From 2736b001d02fc06b3747710986bba25514def3f4 Mon Sep 17 00:00:00 2001 From: uybhatti Date: Fri, 3 Mar 2017 23:55:22 +0100 Subject: [PATCH 1/2] [FLINK-2170] [connectors] Add OrcRowInputFormat and OrcTableSource. --- flink-connectors/flink-orc/pom.xml | 152 + .../org/apache/flink/orc/OrcTableSource.java | 128 + .../java/org/apache/flink/orc/OrcUtils.java | 2229 ++++ .../apache/flink/orc/RowOrcInputFormat.java | 241 + .../flink/orc/OrcTableSourceITCase.java | 142 + .../apache/flink/orc/OrcTableSourceTest.java | 113 + .../flink/orc/RowOrcInputFormatTest.java | 472 + .../test/resources/TestOrcFile.emptyFile.orc | Bin 0 -> 523 bytes .../TestOrcFile.listliststructlong.orc | Bin 0 -> 845 bytes .../test/resources/TestOrcFile.listlong.orc | Bin 0 -> 627 bytes .../test/resources/TestOrcFile.liststring.orc | Bin 0 -> 1298 bytes .../src/test/resources/TestOrcFile.test1.orc | Bin 0 -> 1711 bytes .../resources/TestOrcFile.testDate1900.dat | 10000 ++++++++++++++++ .../resources/TestOrcFile.testDate1900.orc | Bin 0 -> 30941 bytes .../flink-orc/src/test/resources/decimal.dat | 6000 ++++++++++ .../flink-orc/src/test/resources/decimal.orc | Bin 0 -> 16337 bytes .../src/test/resources/demo-11-none.orc | Bin 0 -> 5147970 bytes .../src/test/resources/log4j-test.properties | 27 + flink-connectors/pom.xml | 1 + 19 files changed, 19505 insertions(+) create mode 100644 flink-connectors/flink-orc/pom.xml create mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java create mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java create mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.listliststructlong.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.listlong.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat create mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/decimal.dat create mode 100644 flink-connectors/flink-orc/src/test/resources/decimal.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/demo-11-none.orc create mode 100644 flink-connectors/flink-orc/src/test/resources/log4j-test.properties diff --git a/flink-connectors/flink-orc/pom.xml b/flink-connectors/flink-orc/pom.xml new file mode 100644 index 0000000000000..1ac7eaa7a9f2f --- /dev/null +++ b/flink-connectors/flink-orc/pom.xml @@ -0,0 +1,152 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.4-SNAPSHOT + .. + + + flink-orc_${scala.binary.version} + flink-orc + + jar + + + + + + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + compile + + + + org.apache.flink + flink-streaming-scala_${scala.binary.version} + ${project.version} + compile + + + + org.apache.orc + orc-core + 1.4.0 + + + + + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-clients_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-core + ${project.version} + test + test-jar + + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.maven.plugins + maven-assembly-plugin + [2.4,) + + single + + + + + + + + + org.apache.maven.plugins + maven-clean-plugin + [1,) + + clean + + + + + + + + + org.apache.avro + avro-maven-plugin + [1.7.7,) + + schema + + + + + + + + + + + + + + + diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java new file mode 100644 index 0000000000000..0454ba45af16a --- /dev/null +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.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.flink.orc; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.sources.BatchTableSource; +import org.apache.flink.table.sources.ProjectableTableSource; +import org.apache.flink.table.sources.TableSource; +import org.apache.flink.types.Row; + +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.TypeDescription; + +/** + * Creates a TableSource to read ORC file. + * + *

The ORC file path and schema is passed during {@link OrcTableSource} construction. configuration is optional. + * + *

The OrcTableSource is used as shown in the example below. + * + *

+ * {@code
+ * String path = testInputURL.getPath();
+ * String schema = "struct"
+ * OrcTableSource orcSrc = new OrcTableSource(path, schema);
+ * tEnv.registerTableSource("orcTable", orcSrc);
+ * Table res = tableEnv.sql("SELECT * FROM orcTable");
+ * }
+ * 
+ */ +public class OrcTableSource implements BatchTableSource, ProjectableTableSource { + + private String path; + private TypeDescription orcSchema; + private RowTypeInfo typeInfo; + private Configuration orcConfig; + private int[] fieldMapping; + + /** + * The ORC file path and schema. + * + * @param path the path of orc file + * @param orcSchema schema of orc file + */ + public OrcTableSource(String path, String orcSchema) { + this(path, orcSchema, new Configuration()); + } + + /** + * The file path and schema of orc file, and configuration to read orc file . + * + * @param path the path of orc file + * @param orcSchema schema of orc file + * @param orcConfig configuration to read orc file + */ + public OrcTableSource(String path, String orcSchema, Configuration orcConfig) { + this(path, TypeDescription.fromString(orcSchema), orcConfig); + } + + public OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig) { + this.path = path; + this.orcSchema = orcSchema; + this.orcConfig = orcConfig; + + this.typeInfo = (RowTypeInfo) OrcUtils.schemaToTypeInfo(this.orcSchema); + + } + + @Override + public DataSet getDataSet(ExecutionEnvironment execEnv) { + + RowOrcInputFormat orcIF = new RowOrcInputFormat(path, orcSchema, orcConfig); + if (fieldMapping != null) { + orcIF.setFieldMapping(fieldMapping); + } + return execEnv.createInput(orcIF); + } + + @Override + public TypeInformation getReturnType() { + return typeInfo; + } + + @Override + public TableSource projectFields(int[] fields) { + + OrcTableSource copy = new OrcTableSource(path, orcSchema, orcConfig); + + // set field mapping + copy.fieldMapping = fields; + + // adapt TypeInfo + TypeInformation[] fieldTypes = new TypeInformation[fields.length]; + String[] fieldNames = new String[fields.length]; + for (int i = 0; i < fields.length; i++) { + fieldTypes[i] = this.typeInfo.getTypeAt(fields[i]); + fieldNames[i] = this.typeInfo.getFieldNames()[fields[i]]; + } + copy.typeInfo = new RowTypeInfo(fieldTypes, fieldNames); + + return copy; + } + + @Override + public String explainSource() { + return "ORC Source file at path " + this.path + " with schema " + this.orcSchema; + } + +} diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java new file mode 100644 index 0000000000000..c7557c7477be1 --- /dev/null +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java @@ -0,0 +1,2229 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.types.Row; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; + +import org.apache.orc.TypeDescription; + +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +/** + * A class that provides utility methods for orc file reading. + */ +public class OrcUtils { + + /** + * Convert ORC schema types to Flink types. + * + * @param schema schema of orc file + * + */ + public static TypeInformation schemaToTypeInfo(TypeDescription schema) { + switch (schema.getCategory()) { + case BOOLEAN: + return BasicTypeInfo.BOOLEAN_TYPE_INFO; + case BYTE: + return BasicTypeInfo.BYTE_TYPE_INFO; + case SHORT: + return BasicTypeInfo.SHORT_TYPE_INFO; + case INT: + return BasicTypeInfo.INT_TYPE_INFO; + case LONG: + return BasicTypeInfo.LONG_TYPE_INFO; + case FLOAT: + return BasicTypeInfo.FLOAT_TYPE_INFO; + case DOUBLE: + return BasicTypeInfo.DOUBLE_TYPE_INFO; + case STRING: + case CHAR: + case VARCHAR: + return BasicTypeInfo.STRING_TYPE_INFO; + case DATE: + return SqlTimeTypeInfo.DATE; + case TIMESTAMP: + return SqlTimeTypeInfo.TIMESTAMP; + case BINARY: + return PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO; + case STRUCT: + List fieldSchemas = schema.getChildren(); + TypeInformation[] fieldTypes = new TypeInformation[fieldSchemas.size()]; + for (int i = 0; i < fieldSchemas.size(); i++) { + fieldTypes[i] = schemaToTypeInfo(fieldSchemas.get(i)); + } + String[] fieldNames = schema.getFieldNames().toArray(new String[]{}); + return new RowTypeInfo(fieldTypes, fieldNames); + case LIST: + TypeDescription elementSchema = schema.getChildren().get(0); + TypeInformation elementType = schemaToTypeInfo(elementSchema); + return ObjectArrayTypeInfo.getInfoFor(elementType); + case MAP: + TypeDescription keySchema = schema.getChildren().get(0); + TypeDescription valSchema = schema.getChildren().get(1); + TypeInformation keyType = schemaToTypeInfo(keySchema); + TypeInformation valType = schemaToTypeInfo(valSchema); + return new MapTypeInfo(keyType, valType); + case DECIMAL: + return BasicTypeInfo.BIG_DEC_TYPE_INFO; + case UNION: + throw new UnsupportedOperationException("UNION type not supported yet."); + default: + throw new IllegalArgumentException("Unknown type " + schema); + } + } + + /** + * Fill rows from orc batch. + * + * @param rows the batch of rows need to be filled + * @param schema schema of orc file + * @param batch current orc batch data used to fill the rows + * @param fieldMapping field mapping + * + */ + public static void fillRows(Object[] rows, TypeDescription schema, VectorizedRowBatch batch, int[] fieldMapping) { + + int totalRowsInBatch = (int) batch.count(); + + List fieldTypes = schema.getChildren(); + for (int outIdx = 0; outIdx < fieldMapping.length; outIdx++) { + int inIdx = fieldMapping[outIdx]; + readField(rows, outIdx, fieldTypes.get(inIdx), batch.cols[inIdx], null, Math.min((int) totalRowsInBatch, rows.length)); + } + } + + private static void readField(Object[] rows, int fieldIdx, TypeDescription schema, ColumnVector vector, long[] lengthVector, int childCount) { + + switch (schema.getCategory()) { + case BOOLEAN: + if (vector.noNulls) { + readNonNullBooleanColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readBooleanColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case BYTE: + if (vector.noNulls) { + readNonNullByteColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readByteColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case SHORT: + if (vector.noNulls) { + readNonNullShortColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readShortColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case INT: + if (vector.noNulls) { + readNonNullIntColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readIntColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case LONG: + if (vector.noNulls) { + readNonNullLongColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readLongColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case FLOAT: + if (vector.noNulls) { + readNonNullFloatColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + } else { + readFloatColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + } + break; + case DOUBLE: + if (vector.noNulls) { + readNonNullDoubleColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + } else { + readDoubleColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + } + break; + case CHAR: + case VARCHAR: + case STRING: + if (vector.noNulls) { + readNonNullStringColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + } else { + readStringColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + } + break; + case DATE: + if (vector.noNulls) { + readNonNullDateColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } else { + readDateColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + } + break; + case TIMESTAMP: + if (vector.noNulls) { + readNonNullTimestampColumn(rows, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); + } else { + readTimestampColumn(rows, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); + } + break; + case BINARY: + if (vector.noNulls) { + readNonNullBinaryColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + } else { + readBinaryColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + } + break; + case DECIMAL: + if (vector.noNulls) { + readNonNullDecimalColumn(rows, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); + } + else { + readDecimalColumn(rows, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); + } + break; + case STRUCT: + if (vector.noNulls) { + readNonNullStructColumn(rows, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); + } else { + readStructColumn(rows, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); + } + break; + case LIST: + if (vector.noNulls) { + readNonNullListColumn(rows, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); + } + else { + readListColumn(rows, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); + } + break; + case MAP: + if (vector.noNulls) { + readNonNullMapColumn(rows, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); + } + else { + readMapColumn(rows, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); + } + break; + case UNION: + throw new UnsupportedOperationException("UNION type not supported yet"); + default: + throw new IllegalArgumentException("Unknown type " + schema); + } + } + + private static void readNonNullBooleanColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if boolean is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + boolean repeatingValue = vector.vector[0] != 0; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = vector.vector[i] != 0; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, vector.vector[i] != 0); + } + } + } + } else { // in a list + boolean[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + boolean repeatingValue = vector.vector[0] != 0; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++] != 0; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++] != 0; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullByteColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if byte is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + byte repeatingValue = (byte) vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = (byte) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (byte) vector.vector[i]); + } + } + } + } else { // in a list + byte[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + byte repeatingValue = (byte) vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (byte) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (byte) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullShortColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if short is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + short repeatingValue = (short) vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = (short) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (short) vector.vector[i]); + } + } + } + } else { // in a list + short[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + short repeatingValue = (short) vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (short) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (short) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullIntColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if int is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + int repeatingValue = (int) vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = (int) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (int) vector.vector[i]); + } + } + } + } else { // in a list + int[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + int repeatingValue = (int) vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (int) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (int) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullLongColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if long is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + long repeatingValue = vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (Long) vector.vector[i]); + } + } + } + } else { // in a list + long[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + long repeatingValue = vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullFloatColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + + // check if float is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + float repeatingValue = (float) vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = (float) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, (float) vector.vector[i]); + } + } + } + } else { // in a list + float[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + float repeatingValue = (float) vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (float) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = (float) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullDoubleColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + + // check if double is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + double repeatingValue = vector.vector[0]; + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); + } + } + } + } else { // in a list + double[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + double repeatingValue = vector.vector[0]; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullStringColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + + // check if string is directly in a list or not, e.g, array + if (lengthVector == null) { + if (bytes.isRepeating) { // fill complete column with first value + String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); + fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); + } + } + } + } + else { // in a list + String[] temp; + int offset = 0; + if (bytes.isRepeating) { // fill list with first value + String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); + if (fieldIdx == -1) { // set list as an object + for (int i = 0; i < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field + for (int i = 0; i < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readNonNullDateColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if date is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readDate(vector.vector[0]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[0])); + } + } + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readDate(vector.vector[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[i])); + } + } + } + } else { + Date[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[0]); + } + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[0]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[offset++]); + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[offset++]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullTimestampColumn(Object[] rows, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { + + // check if timestamp is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readTimeStamp(vector.time[0], vector.nanos[0]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readTimeStamp(vector.time[0], vector.nanos[0])); + } + } + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readTimeStamp(vector.time[i], vector.nanos[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readTimeStamp(vector.time[i], vector.nanos[i])); + } + } + } + } else { + Timestamp[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimeStamp(vector.time[0], vector.nanos[0]); + } + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimeStamp(vector.time[0], vector.nanos[0]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimeStamp(vector.time[offset], vector.nanos[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimeStamp(vector.time[offset], vector.nanos[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readNonNullBinaryColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + + // check if string is directly in a list or not, e.g, array + if (lengthVector == null) { + if (bytes.isRepeating) { // fill complete column with first value + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[0], bytes.start[0], bytes.length[0])); + } + } + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); + } + } + } + } else { + byte[][] temp; + int offset = 0; + if (bytes.isRepeating) { // fill complete list with first value + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readNonNullDecimalColumn(Object[] rows, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { + + // check if decimal is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + fillColumnWithRepeatingValue(rows, fieldIdx, readBigDecimal(vector.vector[0]), childCount); + } else { + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = readBigDecimal(vector.vector[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readBigDecimal(vector.vector[i])); + } + } + } + } else { + BigDecimal[] temp; + int offset = 0; + if (vector.isRepeating) { // fill complete list with first value + BigDecimal repeatingValue = readBigDecimal(vector.vector[0]); + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + rows[i] = temp; + offset += temp.length; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + ((Row) rows[i]).setField(fieldIdx, temp); + offset += temp.length; + } + } + } else { + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBigDecimal(vector.vector[offset++]); + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBigDecimal(vector.vector[offset++]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readNonNullStructColumn(Object[] rows, int fieldIdx, StructColumnVector struct, TypeDescription schema, long[] lengthVector, int childCount) { + + List childrenTypes = schema.getChildren(); + + int numChildren = childrenTypes.size(); + Row[] nestedFields = new Row[childCount]; + for (int i = 0; i < childCount; i++) { + nestedFields[i] = new Row(numChildren); + } + for (int i = 0; i < numChildren; i++) { + readField(nestedFields, i, childrenTypes.get(i), struct.fields[i], null, childCount); + } + + // check if struct is directly in a list or not, e.g, array> + if (lengthVector == null) { + if (fieldIdx == -1) { // set struct as an object + System.arraycopy(nestedFields, 0, rows, 0, childCount); + } + else { // set struct as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, nestedFields[i]); + } + } + } + else { // struct in a list + int offset = 0; + Row[] temp; + if (fieldIdx == -1) { // set list of struct as an object + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + System.arraycopy(nestedFields, offset, temp, 0, temp.length); + offset = offset + temp.length; + rows[i] = temp; + } + } + else { // set list of struct as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + System.arraycopy(nestedFields, offset, temp, 0, temp.length); + offset = offset + temp.length; + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static void readNonNullListColumn(Object[] rows, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { + + TypeDescription fieldType = schema.getChildren().get(0); + if (lengthVector == null) { + long[] lengthVectorNested = list.lengths; + readField(rows, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); + } + else { // list in a list + + Object[] nestedList = new Object[childCount]; + + // length vector for nested list + long[] lengthVectorNested = list.lengths; + + // read nested list + readField(nestedList, -1, fieldType, list.child, lengthVectorNested, list.childCount); + + // get type of nestedList + Class classType = nestedList[0].getClass(); + + // fill outer list with nested list + int offset = 0; + int length; + if (fieldIdx == -1) { // set list of list as an object + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + rows[i] = temp; + + } + } else { // set list of list as an field on Row + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + + } + + private static void readNonNullMapColumn(Object[] rows, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { + + List fieldType = schema.getChildren(); + TypeDescription keyType = fieldType.get(0); + TypeDescription valueType = fieldType.get(1); + + ColumnVector keys = map.keys; + ColumnVector values = map.values; + Object[] keyRows = new Object[map.childCount]; + Object[] valueRows = new Object[map.childCount]; + + // read map kes and values + readField(keyRows, -1, keyType, keys, null, keyRows.length); + readField(valueRows, -1, valueType, values, null, valueRows.length); + + // check if map is directly in a list or not, e.g, array> + if (lengthVector == null) { + long[] lengthVectorMap = map.lengths; + int offset = 0; + if (fieldIdx == -1) { + for (int i = 0; i < childCount; i++) { + rows[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); + offset += lengthVectorMap[i]; + } + } else { + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); + offset += lengthVectorMap[i]; + } + } + } else { // list of map + + long[] lengthVectorMap = map.lengths; + int mapOffset = 0; // offset of map element + int offset = 0; // offset of map + HashMap[] temp; + if (fieldIdx == -1) { // set map list as an object + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); + mapOffset += lengthVectorMap[offset]; + offset++; + } + rows[i] = temp; + } + } else { // set map list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); + mapOffset += lengthVectorMap[offset]; + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static void fillColumnWithRepeatingValue(Object[] rows, int fieldIdx, Object repeatingValue, int childCount) { + + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + rows[i] = repeatingValue; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + ((Row) rows[i]).setField(fieldIdx, repeatingValue); + } + } + } + + private static void fillListWithRepeatingNull(Object[] rows, int fieldIdx, Class classType, long[] lengthVector, int childCount) { + + int length; + if (fieldIdx == -1) { + for (int i = 0; i < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + rows[i] = temp; + } + } else { + for (int i = 0; i < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + + private static void readBooleanColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = vector.vector[i] != 0; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, vector.vector[i] != 0); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, boolean[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + boolean[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++] != 0; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new boolean[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++] != 0; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readByteColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = (byte) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, (byte) vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, byte[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + byte[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (byte) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (byte) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readShortColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = (short) vector.vector[i]; + } + } else { // set as field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, (short) vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, short[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + short[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (short) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new short[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (short) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readIntColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = (int) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, (int) vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, int[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + int[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (int) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new int[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (int) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readLongColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, long[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + long[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new long[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readFloatColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = (float) vector.vector[i]; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, (float) vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, float[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + float[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (float) vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; i < childCount; i++) { + temp = new float[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = (float) vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readDoubleColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + + // check if data type(dt) is directly in list or not, e.g, array
+ if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = vector.vector[i]; + } + } else { // set as field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); + } + } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, double[].class, lengthVector, childCount); + } else { + // column contain null values + int offset = 0; + double[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++]; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new double[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + continue; + } + temp[j] = vector.vector[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readStringColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + + // check if string is directly in a list or not, e.g, array + if (lengthVector == null) { + if (bytes.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = bytes.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); + } + } + } + } else { // in a list + if (bytes.isRepeating) { // fill list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, String[].class, lengthVector, childCount); + } else { + int offset = 0; + String[] temp; + boolean[] isNullVector = bytes.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; // skip null value + } + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; // skip null value + } + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readDateColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + + // check if date is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = readDate(vector.vector[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[i])); + } + } + } + } else { + if (vector.isRepeating) { // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, Date[].class, lengthVector, childCount); + } else { + int offset = 0; + Date[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readDate(vector.vector[offset++]); + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readDate(vector.vector[offset++]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + } + + private static void readTimestampColumn(Object[] rows, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { + + // check if timestamp is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + Timestamp ts = new Timestamp(vector.time[i]); + ts.setNanos(vector.nanos[i]); + rows[i] = ts; + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + Timestamp ts = new Timestamp(vector.time[i]); + ts.setNanos(vector.nanos[i]); + ((Row) rows[i]).setField(fieldIdx, ts); + } + } + } + } + else { + if (vector.isRepeating) { // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, Timestamp[].class, lengthVector, childCount); + } else { + int offset = 0; + Timestamp[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = new Timestamp(vector.time[offset]); + temp[j].setNanos(vector.nanos[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = new Timestamp(vector.time[offset]); + temp[j].setNanos(vector.nanos[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readBinaryColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + + // check if string is directly in a list or not, e.g, array + if (lengthVector == null) { + if (bytes.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVectorIndex = bytes.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVectorIndex[i]) { + rows[i] = null; + continue; + } + rows[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVectorIndex[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); + } + } + } + } else { + if (bytes.isRepeating) { // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, byte[][].class, lengthVector, childCount); + } else { + int offset = 0; + byte[][] temp; + boolean[] isNullVector = bytes.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + rows[i] = temp; + } + } else { // set list as a field + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readDecimalColumn(Object[] rows, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { + + // check if decimal is directly in a list or not, e.g, array + if (lengthVector == null) { + if (vector.isRepeating) { // fill complete column with first value + // Also column contains null value and it's repeating + fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + } else { + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = readBigDecimal(vector.vector[i]); + } + } else { // set as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, readBigDecimal(vector.vector[i])); + } + } + } + } else { + if (vector.isRepeating) { // fill complete list with first value + // Also column contains null value and it's repeating + // so all values are null, but we need to set list with null values + fillListWithRepeatingNull(rows, fieldIdx, BigDecimal[].class, lengthVector, childCount); + } else { + int offset = 0; + BigDecimal[] temp; + boolean[] isNullVector = vector.isNull; + if (fieldIdx == -1) { // set list as an object + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readBigDecimal(vector.vector[offset++]); + } + rows[i] = temp; + } + } else { // set list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + temp[j] = null; + continue; + } + temp[j] = readBigDecimal(vector.vector[offset++]); + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + } + + private static void readStructColumn(Object[] rows, int fieldIdx, StructColumnVector struct, TypeDescription schema, long[] lengthVector, int childCount) { + + List childrenTypes = schema.getChildren(); + + int numChildren = childrenTypes.size(); + Row[] nestedFields = new Row[childCount]; + for (int i = 0; i < childCount; i++) { + nestedFields[i] = new Row(numChildren); + } + for (int i = 0; i < numChildren; i++) { + readField(nestedFields, i, childrenTypes.get(i), struct.fields[i], null, childCount); + } + + boolean[] isNullVector = struct.isNull; + + // check if struct is directly in a list or not, e.g, array> + if (lengthVector == null) { + if (fieldIdx == -1) { // set struct as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = nestedFields[i]; + } + } else { // set struct as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, nestedFields[i]); + } + } + } else { // struct in a list + int offset = 0; + Row[] temp; + if (fieldIdx == -1) { // set list of struct as an object + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + continue; + } + temp[j] = nestedFields[offset++]; + } + rows[i] = temp; + } + } + else { // set list of struct as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + continue; + } + temp[j] = nestedFields[offset++]; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static void readListColumn(Object[] rows, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { + + TypeDescription fieldType = schema.getChildren().get(0); + if (lengthVector == null) { + long[] lengthVectorNested = list.lengths; + readField(rows, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); + } + else { // list in a list + + Object[] nestedList = new Object[childCount]; + + // length vector for nested list + long[] lengthVectorNested = list.lengths; + + // read nested list + readField(nestedList, -1, fieldType, list.child, lengthVectorNested, list.childCount); + + // get type of nestedList + Class classType = nestedList[0].getClass(); + + // fill outer list with nested list + int offset = 0; + int length; + if (fieldIdx == -1) { // set list of list as an object + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + rows[i] = temp; + + } + } else { // set list of list as an field on Row + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object temp = Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static void readMapColumn(Object[] rows, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { + + List fieldType = schema.getChildren(); + TypeDescription keyType = fieldType.get(0); + TypeDescription valueType = fieldType.get(1); + + ColumnVector keys = map.keys; + ColumnVector values = map.values; + Object[] keyRows = new Object[map.childCount]; + Object[] valueRows = new Object[map.childCount]; + + // read map kes and values + readField(keyRows, -1, keyType, keys, null, keyRows.length); + readField(valueRows, -1, valueType, values, null, valueRows.length); + + boolean[] isNullVector = map.isNull; + + // check if map is directly in a list or not, e.g, array> + if (lengthVector == null) { + long[] lengthVectorMap = map.lengths; + int offset = 0; + if (fieldIdx == -1) { // set map as an object + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + rows[i] = null; + continue; + } + rows[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); + offset += lengthVectorMap[i]; + } + } else { // set map as a field of Row + for (int i = 0; i < childCount; i++) { + if (isNullVector[i]) { + ((Row) rows[i]).setField(fieldIdx, null); + continue; + } + ((Row) rows[i]).setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); + offset += lengthVectorMap[i]; + } + } + } else { // list of map + long[] lengthVectorMap = map.lengths; + int mapOffset = 0; // offset of map element + int offset = 0; // offset of map + HashMap[] temp; + if (fieldIdx == -1) { // set map list as an object + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + continue; + } + temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); + mapOffset += lengthVectorMap[offset]; + offset++; + } + rows[i] = temp; + } + } else { // set map list as a field of Row + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + continue; + } + temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); + mapOffset += lengthVectorMap[offset]; + offset++; + } + ((Row) rows[i]).setField(fieldIdx, temp); + } + } + } + } + + private static BigDecimal readBigDecimal(HiveDecimalWritable hiveDecimalWritable) { + HiveDecimal hiveDecimal = hiveDecimalWritable.getHiveDecimal(); + return hiveDecimal.bigDecimalValue(); + } + + private static byte[] readBinary(byte[] src, int srcPos, int length) { + byte[] result = new byte[length]; + System.arraycopy(src, srcPos, result, 0, length); + return result; + } + + private static Timestamp readTimeStamp(long time, int nanos) { + Timestamp ts = new Timestamp(time); + ts.setNanos(nanos); + return ts; + } + + private static Date readDate(long days) { + // day to milliseconds + return new Date(days * 24 * 60 * 60 * 1000); + } + + private static HashMap readHashMap(Object[] keyRows, Object[] valueRows, int offset, long length) { + + HashMap resultMap = new HashMap<>(); + for (int j = 0; j < length; j++) { + resultMap.put(keyRows[offset], valueRows[offset]); + offset++; + } + return resultMap; + } + +} diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java new file mode 100644 index 0000000000000..0c9c549aed2cf --- /dev/null +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java @@ -0,0 +1,241 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.io.FileInputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; + +import org.apache.orc.OrcConf; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.RecordReader; +import org.apache.orc.TypeDescription; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +import static org.apache.flink.orc.OrcUtils.fillRows; + +/** + * InputFormat to read ORC data. + * For Optimization, reading is done in batch instead of a single row. + */ +public class RowOrcInputFormat + extends FileInputFormat + implements ResultTypeQueryable { + + private static final Logger LOG = LoggerFactory.getLogger(RowOrcInputFormat.class); + private static final int BATCH_SIZE = 1024; + + private org.apache.hadoop.conf.Configuration config; + private TypeDescription schema; + private int[] fieldMapping; + + private transient RowTypeInfo rowType; + private transient RecordReader orcRowsReader; + private transient VectorizedRowBatch rowBatch; + private transient Row[] rows; + + private transient int rowInBatch; + + public RowOrcInputFormat(String path, String schemaString, Configuration orcConfig) { + this(path, TypeDescription.fromString(schemaString), orcConfig); + } + + public RowOrcInputFormat(String path, TypeDescription orcSchema, Configuration orcConfig) { + super(new Path(path)); + this.unsplittable = false; + this.schema = orcSchema; + this.rowType = (RowTypeInfo) OrcUtils.schemaToTypeInfo(schema); + this.config = orcConfig; + + this.fieldMapping = new int[this.schema.getChildren().size()]; + for (int i = 0; i < fieldMapping.length; i++) { + this.fieldMapping[i] = i; + } + + } + + public void setFieldMapping(int[] fieldMapping) { + this.fieldMapping = fieldMapping; + // adapt result type + + TypeInformation[] fieldTypes = new TypeInformation[fieldMapping.length]; + String[] fieldNames = new String[fieldMapping.length]; + for (int i = 0; i < fieldMapping.length; i++) { + fieldTypes[i] = this.rowType.getTypeAt(fieldMapping[i]); + fieldNames[i] = this.rowType.getFieldNames()[fieldMapping[i]]; + } + this.rowType = new RowTypeInfo(fieldTypes, fieldNames); + } + + private boolean[] computeProjectionMask() { + boolean[] projectionMask = new boolean[schema.getMaximumId() + 1]; + for (int inIdx : fieldMapping) { + TypeDescription fieldSchema = schema.getChildren().get(inIdx); + for (int i = fieldSchema.getId(); i <= fieldSchema.getMaximumId(); i++) { + projectionMask[i] = true; + } + } + return projectionMask; + } + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + this.rows = new Row[BATCH_SIZE]; + for (int i = 0; i < BATCH_SIZE; i++) { + rows[i] = new Row(fieldMapping.length); + } + } + + @Override + public void open(FileInputSplit fileSplit) throws IOException { + + this.currentSplit = fileSplit; + Preconditions.checkArgument(this.splitStart == 0, "ORC files must be read from the start."); + + if (LOG.isDebugEnabled()) { + LOG.debug("Opening ORC file " + fileSplit.getPath()); + } + + org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(fileSplit.getPath().getPath()); + + Reader orcReader = OrcFile.createReader(hPath, OrcFile.readerOptions(config)); + + Reader.Options options = orcReader.options() + .range(fileSplit.getStart(), fileSplit.getLength()) + .useZeroCopy(OrcConf.USE_ZEROCOPY.getBoolean(config)) + .skipCorruptRecords(OrcConf.SKIP_CORRUPT_DATA.getBoolean(config)) + .tolerateMissingSchema(OrcConf.TOLERATE_MISSING_SCHEMA.getBoolean(config)); + + options.include(computeProjectionMask()); + + // check that schema of file is as expected + if (!this.schema.equals(orcReader.getSchema())) { + + throw new RuntimeException("Invalid schema for file at " + this.filePath + + " Expected:" + this.schema + " Actual: " + orcReader.getSchema()); + } + + this.orcRowsReader = orcReader.rows(options); + + // assign ids + this.schema.getId(); + + this.rowBatch = schema.createRowBatch(BATCH_SIZE); + rowInBatch = 0; + } + + @Override + public void close() throws IOException { + + if (orcRowsReader != null) { + this.orcRowsReader.close(); + } + this.orcRowsReader = null; + + } + + @Override + public void closeInputFormat() throws IOException { + this.rows = null; + this.rows = null; + this.schema = null; + this.rowBatch = null; + } + + @Override + public boolean reachedEnd() throws IOException { + return !ensureBatch(); + } + + private boolean ensureBatch() throws IOException { + + if (rowInBatch >= rowBatch.size) { + rowInBatch = 0; + boolean moreRows = orcRowsReader.nextBatch(rowBatch); + + if (moreRows) { + // read rows + fillRows(rows, schema, rowBatch, fieldMapping); + } + return moreRows; + } + + return true; + } + + @Override + public Row nextRecord(Row reuse) throws IOException { + return rows[this.rowInBatch++]; + } + + @Override + public TypeInformation getProducedType() { + return rowType; + } + + // -------------------------------------------------------------------------------------------- + // Custom serialization methods + // -------------------------------------------------------------------------------------------- + + private void writeObject(ObjectOutputStream out) throws IOException { + this.config.write(out); + out.writeUTF(schema.toString()); + + out.writeInt(fieldMapping.length); + for (int f : fieldMapping) { + out.writeInt(f); + } + + } + + @SuppressWarnings("unchecked") + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + + org.apache.hadoop.conf.Configuration configuration = new org.apache.hadoop.conf.Configuration(); + configuration.readFields(in); + + if (this.config == null) { + this.config = configuration; + } + this.schema = TypeDescription.fromString(in.readUTF()); + + this.fieldMapping = new int[in.readInt()]; + for (int i = 0; i < fieldMapping.length; i++) { + this.fieldMapping[i] = in.readInt(); + } + + } + +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java new file mode 100644 index 0000000000000..3de6ab31010a5 --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java @@ -0,0 +1,142 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.java.BatchTableEnvironment; +import org.apache.flink.test.util.MultipleProgramsTestBase; +import org.apache.flink.types.Row; + +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; + +import java.net.URL; +import java.util.ArrayList; +import java.util.List; + +/** + * Tests for {@link OrcTableSource}. + */ +public class OrcTableSourceITCase extends MultipleProgramsTestBase { + + private static final String TEST1_SCHEMA = "struct>>," + + "list:array>," + + "map:map>>"; + + private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); + + + private static final String[] TEST1_DATA = new String[] { + "false,1,1024,65536,9223372036854775807,1.0,-15.0,[0, 1, 2, 3, 4],hi,[1,bye, 2,sigh],[3,good, 4,bad],{}", + "true,100,2048,65536,9223372036854775807,2.0,-5.0,[],bye,[1,bye, 2,sigh]," + + "[100000000,cat, -100000,in, 1234,hat],{chani=5,chani, mauddib=1,mauddib}" }; + + public OrcTableSourceITCase() { + super(TestExecutionMode.COLLECTION); + } + + @Test + public void testOrcTableSource() throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); + + assert (test1URL != null); + OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + + tEnv.registerTableSource("orcTable", orc); + + String query = "Select * from orcTable"; + Table t = tEnv.sql(query); + + DataSet dataSet = tEnv.toDataSet(t, Row.class); + List records = dataSet.collect(); + + Assert.assertEquals(records.size(), 2); + + List actualRecords = new ArrayList<>(); + for (Row record : records) { + Assert.assertEquals(record.getArity(), 12); + actualRecords.add(record.toString()); + } + + Assert.assertThat(actualRecords, CoreMatchers.hasItems(TEST1_DATA)); + } + + @Test + public void testOrcTableProjection() throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); + + assert(test1URL != null); + OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + + tEnv.registerTableSource("orcTable", orc); + + String query = "Select middle,list,map from orcTable"; + Table t = tEnv.sql(query); + + String[] colNames = new String[] {"middle", "list", "map"}; + + RowTypeInfo rowTypeInfo = new RowTypeInfo( + new TypeInformation[] { + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}, + new String[] {"int1", "string1"}); + + RowTypeInfo structTypeInfo = new RowTypeInfo( + new TypeInformation[] {ObjectArrayTypeInfo.getInfoFor(rowTypeInfo)}, + new String[] {"list"}); + + TypeInformation[] colTypes = new TypeInformation[] { + structTypeInfo, + ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), + new MapTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, rowTypeInfo) + }; + + TableSchema actualTableSchema = new TableSchema(colNames, colTypes); + + Assert.assertArrayEquals(t.getSchema().getColumnNames(), colNames); + Assert.assertArrayEquals(t.getSchema().getTypes(), colTypes); + Assert.assertEquals(actualTableSchema.toString(), t.getSchema().toString()); + + DataSet dataSet = tEnv.toDataSet(t, Row.class); + List records = dataSet.collect(); + + Assert.assertEquals(records.size(), 2); + for (Row record: records) { + Assert.assertEquals(record.getArity(), 3); + } + + } + +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java new file mode 100644 index 0000000000000..c285054c6e155 --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java @@ -0,0 +1,113 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.java.BatchTableEnvironment; + +import org.junit.Assert; +import org.junit.Test; + +import java.net.URL; + +/** + * Unit Tests for {@link OrcTableSource}. + */ +public class OrcTableSourceTest { + + private static final String TEST1_SCHEMA = "struct>>," + + "list:array>," + + "map:map>>"; + + private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); + + @Test + public void testOrcSchema() throws Exception { + + assert(test1URL != null); + OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + + String expectedSchema = "Row(boolean1: Boolean, byte1: Byte, short1: Short, int1: Integer, long1: Long, " + + "float1: Float, double1: Double, bytes1: byte[], string1: String, " + + "middle: Row(list: ObjectArrayTypeInfo), " + + "list: ObjectArrayTypeInfo, " + + "map: Map)"; + + Assert.assertEquals(expectedSchema, orc.getReturnType().toString()); + + } + + @Test + public void testOrcTableSchema() throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); + + assert(test1URL != null); + OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + + tEnv.registerTableSource("orcTable", orc); + String query = "Select * from orcTable"; + Table t = tEnv.sql(query); + + String[] colNames = new String[] { + "boolean1", "byte1", "short1", "int1", "long1", "float1", + "double1", "bytes1", "string1", "list", "list0", "map" + }; + + RowTypeInfo rowTypeInfo = new RowTypeInfo( + new TypeInformation[] { + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}, + new String[] {"int1", "string1"}); + + TypeInformation[] colTypes = new TypeInformation[] { + BasicTypeInfo.BOOLEAN_TYPE_INFO, + BasicTypeInfo.BYTE_TYPE_INFO, + BasicTypeInfo.SHORT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.FLOAT_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), + ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), + new MapTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, rowTypeInfo) + }; + TableSchema expectedTableSchema = new TableSchema(colNames, colTypes); + + Assert.assertArrayEquals(t.getSchema().getColumnNames(), colNames); + Assert.assertArrayEquals(t.getSchema().getTypes(), colTypes); + Assert.assertEquals(expectedTableSchema.toString(), t.getSchema().toString()); + + } + +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java new file mode 100644 index 0000000000000..60008a0838612 --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java @@ -0,0 +1,472 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.types.Row; + +import org.apache.hadoop.conf.Configuration; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Tests for the {@link RowOrcInputFormat}. + */ + +public class RowOrcInputFormatTest { + + private RowOrcInputFormat rowOrcInputFormat; + + @After + public void tearDown() throws IOException { + if (rowOrcInputFormat != null) { + rowOrcInputFormat.close(); + rowOrcInputFormat.closeInputFormat(); + } + rowOrcInputFormat = null; + } + + private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); + + private static final String TEST1_SCHEMA = "struct>>," + + "list:array>," + + "map:map>>"; + + private static final String[] TEST1_DATA = new String[] { + "false,1,1024,65536,9223372036854775807,1.0,-15.0,[0, 1, 2, 3, 4],hi,[1,bye, 2,sigh],[3,good, 4,bad],{}", + "true,100,2048,65536,9223372036854775807,2.0,-5.0,[],bye,[1,bye, 2,sigh]," + + "[100000000,cat, -100000,in, 1234,hat],{chani=5,chani, mauddib=1,mauddib}" }; + + private static final String[] TEST1_PROJECTED_DATA = new String[] { + "{},[3,good, 4,bad],[1,bye, 2,sigh],hi,[0, 1, 2, 3, 4],-15.0,1.0,9223372036854775807,65536,1024,1,false", + "{chani=5,chani, mauddib=1,mauddib},[100000000,cat, -100000,in, 1234,hat],[1,bye, 2,sigh],bye," + + "[],-5.0,2.0,9223372036854775807,65536,2048,100,true" }; + + private static final String TEST1_INVALID_SCHEMA = "struct>>," + + "list:array>," + + "map:map>>"; + + @Test(expected = FileNotFoundException.class) + public void testInvalidPath() throws IOException{ + + rowOrcInputFormat = new RowOrcInputFormat("TestOrcFile.test2.orc", TEST1_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.open(inputSplits[0]); + + } + + @Test(expected = RuntimeException.class) + public void testInvalidSchema() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_INVALID_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.open(inputSplits[0]); + + } + + @Test(expected = IndexOutOfBoundsException.class) + public void testInvalidProjection() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + int[] projectionMask = {14}; + rowOrcInputFormat.setFieldMapping(projectionMask); + } + + @Test + public void testMajorDataTypes() throws IOException{ + + // test for boolean,byte,short,int,long,float,double,bytes,string,struct,list,map + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.toString(), TEST1_DATA[count++]); + } + } + } + + @Test + public void testProjection() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + int[] projectionMask = {11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0}; + rowOrcInputFormat.setFieldMapping(projectionMask); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.toString(), TEST1_PROJECTED_DATA[count++]); + } + } + + } + + @Test + public void testTimeStampAndDate() throws IOException{ + + URL expectedDataURL = getClass().getClassLoader().getResource("TestOrcFile.testDate1900.dat"); + assert(expectedDataURL != null); + List expectedTimeStampAndDate = Files.readAllLines(Paths.get(expectedDataURL.getPath())); + + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.testDate1900.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct"; + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + List actualTimeStampAndDate = new ArrayList<>(); + + Row row = null; + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + count++; + if (count <= 10000) { + actualTimeStampAndDate.add(row.getField(0) + "," + row.getField(1)); + } + + } + } + Assert.assertEquals(count, 70000); + Assert.assertEquals(expectedTimeStampAndDate.size(), actualTimeStampAndDate.size()); + Assert.assertEquals(expectedTimeStampAndDate.toString(), actualTimeStampAndDate.toString()); + + } + + @Test + public void testDecimal() throws IOException{ + + URL expectedDataURL = getClass().getClassLoader().getResource("decimal.dat"); + List expectedDecimal = Files.readAllLines(Paths.get(expectedDataURL.getPath())); + + URL testInputURL = getClass().getClassLoader().getResource("decimal.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct<_col0:decimal(10,5)>"; + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + List actualDecimal = new ArrayList<>(); + + Row row = null; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + actualDecimal.add(row.getField(0)); + } + } + + Assert.assertEquals(expectedDecimal.size(), actualDecimal.size()); + Assert.assertEquals(expectedDecimal.toString(), actualDecimal.toString()); + + } + + @Test + public void testEmptyFile() throws IOException{ + + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.emptyFile.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + + rowOrcInputFormat = new RowOrcInputFormat(path, TEST1_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = new Row(1); + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + count++; + } + } + + Assert.assertEquals(count, 0); + } + + @Test + public void testLargeFile() throws IOException{ + + URL testInputURL = getClass().getClassLoader().getResource("demo-11-none.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int," + + "_col5:string,_col6:int,_col7:int,_col8:int>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = new Row(1); + int count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + count++; + } + } + + Assert.assertEquals(count, 1920800); + } + + @Test + public void testProducedType() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + rowOrcInputFormat.open(inputSplits[0]); + + TypeInformation type = rowOrcInputFormat.getProducedType(); + Assert.assertEquals(type.toString(), "Row(boolean1: Boolean, byte1: Byte, short1: Short, int1: Integer," + + " long1: Long, float1: Float, double1: Double, bytes1: byte[], string1: String," + + " middle: Row(list: ObjectArrayTypeInfo)," + + " list: ObjectArrayTypeInfo," + + " map: Map)"); + + } + + @Test + public void testProducedTypeWithProjection() throws IOException{ + + assert(test1URL != null); + rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); + int[] projectionMask = {9, 10, 11}; + rowOrcInputFormat.setFieldMapping(projectionMask); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + rowOrcInputFormat.open(inputSplits[0]); + + TypeInformation type = rowOrcInputFormat.getProducedType(); + Assert.assertEquals(type.toString(), "Row(middle: Row(list: ObjectArrayTypeInfo)," + + " list: ObjectArrayTypeInfo," + + " map: Map)"); + + } + + @Test + public void testLongList() throws Exception { + + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.listlong.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + long count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.getArity(), 1); + Object object = row.getField(0); + long[] l = (long[]) object; + + Assert.assertEquals(l.length, 2); + if (count < 50) { + Assert.assertArrayEquals(l, new long[]{count, count + 1}); + } + else { + Assert.assertArrayEquals(l, new long[]{0L, 0L}); + } + count = count + 2; + } + } + Assert.assertEquals(count, 100); + } + + @Test + public void testStringList() throws Exception { + + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.liststring.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + long count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.getArity(), 1); + Object object = row.getField(0); + String[] l = (String[]) object; + + Assert.assertEquals(l.length, 2); + Assert.assertArrayEquals(l, new String[]{"hello" + count, "hello" + (count + 1) }); + count = count + 2; + } + } + Assert.assertEquals(count, 200); + } + + @Test + public void testListOfListOfStructOfLong() throws Exception { + URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.listliststructlong.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct>>>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + + Assert.assertEquals(inputSplits.length, 1); + + Row row = null; + long count = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + while (!rowOrcInputFormat.reachedEnd()) { + + row = rowOrcInputFormat.nextRecord(row); + Assert.assertEquals(row.getArity(), 1); + + Object[] objects = (Object[]) row.getField(0); + Assert.assertEquals(objects.length, 1); + + Object[] objects1 = (Object[]) objects[0]; + Assert.assertEquals(objects1.length, 1); + + Row[] nestedRows = Arrays.copyOf(objects1, objects1.length, Row[].class); + Assert.assertEquals(nestedRows.length, 1); + + Assert.assertEquals(nestedRows[0].getArity(), 1); + + Assert.assertEquals(nestedRows[0].getField(0), count); + + count++; + } + } + Assert.assertEquals(count, 100); + } + + @Test + public void testSplit() throws IOException{ + + URL testInputURL = getClass().getClassLoader().getResource("demo-11-none.orc"); + assert(testInputURL != null); + String path = testInputURL.getPath(); + String schema = "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int," + + "_col5:string,_col6:int,_col7:int,_col8:int>"; + + rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); + rowOrcInputFormat.openInputFormat(); + + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(10); + + Assert.assertEquals(inputSplits.length, 10); + + Row row = null; + int countTotalRecords = 0; + for (FileInputSplit split : inputSplits) { + rowOrcInputFormat.open(split); + int countSplitRecords = 0; + while (!rowOrcInputFormat.reachedEnd()) { + row = rowOrcInputFormat.nextRecord(row); + countSplitRecords++; + } + Assert.assertNotEquals(countSplitRecords, 1920800); + countTotalRecords += countSplitRecords; + } + + Assert.assertEquals(countTotalRecords, 1920800); + } + +} diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc new file mode 100644 index 0000000000000000000000000000000000000000..ecdadcbff134615d7eefcb740d55fe710cee059b GIT binary patch literal 523 zcmaix$xg#C5Qe>EnJmx{S{5I3LY$gwPl(Ki0VfCx86O*nAn33yj{0RALJfJ%g= z-yY9=v&depmn0*ZJW?2kTu94yT=M+Mv7+18fd!+++rKoYVb8*(6m+$6w2JQ%VP zoBWuE$qYVmaJZY`s{`<^gga2tKpn+nltV!*rn5eyFL-h~g3j5(B6iCb91v J8R&e0I%!PbNv8+gamelvDCX%*5(l1yh8mvCbhTm+DWG ztwa_dU#Z@DIhV8B;OhB~-R0@ul$5?`y7+ZIH($|mqj$k$U8AmcUDj$pnUI%_5|g`{ z_pu*k`hH3v+BCv+irL?_?^!%_DtEq7PmP?caWOG^s&}!BC1(=&Qg19mLDn{_(&)5@POtR&tadvR9xSX{NJBb%7= zt4}R)n|QOUwU+;q*euNVx90xK%Z_|`W-cOGt!Dr3c^5Rx+Ua`FUijqE)I)rq=R7`g zH81qX@ei{tH+_qD-ms;qt}$#=>vH zo3i@)wRdg~T6DH-QB3@q2N{blifa8lXk2r0E7z_EGHI^AbR}Yc7b^ZTJC>R<@7R;f z(~G7?F<){0!_LXTa3GnH0hwZ8n!v!|+ilI);K1QB-9Mt~?6G5=Yumr=Qc^sY7I)8} zW#`&P-7~h8M_#vt`t3NibMEb>-!D}kl{Ahm^cI`bzxjmgsl#`!vTIDP|7pr^226;T z9wa^V;7mB+^r>(OM`3{hhfMO6h{w_g7#P+8WfR119M?K}_MlF|Y*(j{gVD;ao&C-3 z9TNk)Y_}V!Z{BevpmS=ocgI9dCqw6y$OD^xR=#ZW@{!SD(+ty*(PZR_@UZI>QRL%e pYfSDD%C2FSkcr#C%)k*Pz$nq6pv1($qakO&*7Sv$*+0lx3;+xlS?&M; literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.listlong.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.listlong.orc new file mode 100644 index 0000000000000000000000000000000000000000..648ea18084ed061f03afc0e6453778c1d617d280 GIT binary patch literal 627 zcmV-(0*w7nQbQ2{015^Y1PC%v04M+eV7f4AZ$rdMGkni%gxIAM^?8)@NzR1G03_ zQ~I=$l8)`w(zKm!g}HebqmEB z$(}8Vks5ovjmb^kX-N*YbktC-?e0-&YOo*RHb1o4jT#G6Uq?2aIh-Ryw8c%=ROlA(7bVdZ!>jqI83wO`#jQ!BZ1BcvtYw$6ASzM8ou-0iYb#i<<5?Y8 zcR86{`PbuAyu-G0%+|g?R_Q}}d}YnIwm1mjJj`x;n`S?P$jk3QlgcznjWaUDr?(61 z3Bbqtmr$)JCquHRII7bMwFr!QAr#AIzr0d}oOz2A#o{B}df>n)gR+|a0pW23?LA#& zcAvMyU=0E<{s9R90KhQ<01*HHz%c*h0OOM8V&O0fV1Sam z94SKV91H?U5|e}j7yy(20OMd5_{=QD&A}|NfLTIF!A`?SiI0Ovh>=l>J-0F^v$(`i ziIsy(h>=N&iG#(!$clx-D1ZS<@^YjIv2!p8BuPvX4q$M2!43cjO%MSXfFL3Q01PM` NFb0D30|QS|Llz^%2`T^p literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc new file mode 100644 index 0000000000000000000000000000000000000000..75a5f2a10a3fe703394fd3700e56d06149f2785b GIT binary patch literal 1298 zcmV+t1?~DzQbQ2{015^Y1PEkM04M+e|ol`QeugC00RIb1OOdU!HN?>5PjWgV`Yg@6M_T} zQj^_<=xJ8=;;j=RGjzO2ys!ex3fVHFMPf}N0~_|J$b(oPe=BEqIe z@P+Q{3BlH(q^KS_Q66s*qbnS;<%}RL3%{1=BQ6o&kDZt%K`b7imB4z^u%)5M`$3|1 z!$5h0e8-8HVU#(PcXGQlE7Kax-Uj?& z!#wOIPOfEmByAH>txpV$-a8&5P(_YG26PHYo!MV84}7~>7RQl&sAf< zk-72I{bY7VtVnN>nrTZf<~#-2&t3rAgjOZ}g{>kyvM5fFwQb86006*a0stBV z06o(|uADFw1<@^07-PJkNPiG1`=3RjicH>&qi(4C^XvETKgZ9%%X`oFj`v>go$tNh zyWab}cfXe&_Gx?^pGN=%B+x(vl~d*HH>jK{r^=~vs+=mP%BeC`hRW!#s0@{%GE|1j zP#G$h%B6CtT)h#MOXX6zR4$cEWvWb-sWMe&FIHu$OqHoJRc@7AZk4<5qjIa< zD!0l~St?6qsVtSHvii&_OJ%7%Dv!#e@~Av2kIJL+REx@^vQ@UqR@o|BWvgtJt+G|N z3Wp4b4u=qj5{DFr7Ka#z8iyQ5-v5mAzL39?zm&h0znH(8zntrtf~KN5`+YS{QB&2F zHFZs4Q`wX@wM}s|`oHB(eN*66I3-SvQ{+@RWlo)Q_5LcIQm57_cB-9nr`{=eDxQ*O z_C1Q8s;BI!dkUY*r}U|Pil6G|?)%n14S*Iv6QB*y2xtW~1KI%%fmJU}fwn+npf%7O zXb&_9S_DmkHo;SWt%7DjyP#pvGH4pK4H^fngXY2Be*qN$00#!ZSqA{XV+a6%001?K zK?;CC36VJHcrhQ)_j>Tb#)6j#Gf5Ahjca;@_Q=nTv)p)8Qgd?h4TRXiw56rQ67c{A0IUE2 z<6su}%^}6Z!7MP5RbmOVg0n`75+4VT5F?`$dv0Y;W^sw35-SIn5F?Wk69>CNiWLh- zN&o|tl;$|Ws3gM0mXVs1lW!oz4yG+FC6FgUzm2LK3J5CIr~AR+<)3@9fs27>eh I15Z*z7I1J@L;wH) literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc new file mode 100644 index 0000000000000000000000000000000000000000..4fb0beff868971efb653739fe6ae47a37e4a1c66 GIT binary patch literal 1711 zcma)53s6%>6y2AX#Fs$e5wL~=SI1j7^ppUN+?iL7eMX!KP?|U(2PGykr zMnDEm3tYcYD8Tt@fkL4XffW78WSpFmVtur#7q0|w$w2R*$@rIbh!<9^<^mL0Lk+rG z7y%GVJw2ukQ|}5C{s@8!l@@f?W3G^{))X}OTNY*dP!`yK*kA&C*IDdcYs=phkq172 zl;I+!ixKH$=mn9X%L9%Oe|Hf##E4cjDk@BfpcCUH;z))m5B!%Ri3irhN|VJ(Ba8$c zuJ~RTmmMt)WsJ?{hw@(UIzZdC(#W!?aRcQS*!7#jZVTou4o-nHTCV#KW zY{|-Mw5FuLKI&`p5VZ497v}k2IwSz*xk~_eg@|YXgdiBe7i^Rw1OVVq2jF7(tUfX? zU@yP`$Tq>iWdIN^0LUFwkm0NbKnG$70VP5mcZ3H|krHSfJh>oc+L%Xy3~Fr+4_uU| zO=If=0FxHnT})gBzyTr@BEyC04BZ6)rNPT61?B*VqoYN_Fj08eeu*?PY(Kogo#?uj zr7g5DhRsBVNhN#3;zc5fP^%uOjrnh-!L9pc6=7+7%vM9Q$0swq zYndaP)cv+kU8bw-28{y;gN3drVcdBz9t7dhuMmb;CnuK$&m)N*ABg3G{t2zI3h)7JHQ`w$?;Yul6&u ziwZDH8s)Zzzn)e#+n)4aslGibo1Dd}Z!(O7*z|E%ly3M1cAf zYt1z)_kqG1Uqzcg?G&q_?Z#etp>J!{dl`+{7n_4>OPBj?lslX5vp~G+Qn9goge**% z^tY~Wx04^}8SVY?Oi#J%6t%R_-}~zcW<%i-&9}MC=;kgDljiWlR`QBJa-XMfic6|J zaZJ9)>Wb5cGce^ly-Ls2ct8(Y?Re=hf9M( zLo**0tb4TQEc>Otekshx*T)2S*OK9$LOJF>!JUR*2#VMT+3 literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat new file mode 100644 index 0000000000000..59b933ddf0b5c --- /dev/null +++ b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat @@ -0,0 +1,10000 @@ +1900-05-05 12:34:56.1,1900-12-25 +1900-05-05 12:34:56.1001,1900-12-25 +1900-05-05 12:34:56.1002,1900-12-25 +1900-05-05 12:34:56.1003,1900-12-25 +1900-05-05 12:34:56.1004,1900-12-25 +1900-05-05 12:34:56.1005,1900-12-25 +1900-05-05 12:34:56.1006,1900-12-25 +1900-05-05 12:34:56.1007,1900-12-25 +1900-05-05 12:34:56.1008,1900-12-25 +1900-05-05 12:34:56.1009,1900-12-25 +1900-05-05 12:34:56.101,1900-12-25 +1900-05-05 12:34:56.1011,1900-12-25 +1900-05-05 12:34:56.1012,1900-12-25 +1900-05-05 12:34:56.1013,1900-12-25 +1900-05-05 12:34:56.1014,1900-12-25 +1900-05-05 12:34:56.1015,1900-12-25 +1900-05-05 12:34:56.1016,1900-12-25 +1900-05-05 12:34:56.1017,1900-12-25 +1900-05-05 12:34:56.1018,1900-12-25 +1900-05-05 12:34:56.1019,1900-12-25 +1900-05-05 12:34:56.102,1900-12-25 +1900-05-05 12:34:56.1021,1900-12-25 +1900-05-05 12:34:56.1022,1900-12-25 +1900-05-05 12:34:56.1023,1900-12-25 +1900-05-05 12:34:56.1024,1900-12-25 +1900-05-05 12:34:56.1025,1900-12-25 +1900-05-05 12:34:56.1026,1900-12-25 +1900-05-05 12:34:56.1027,1900-12-25 +1900-05-05 12:34:56.1028,1900-12-25 +1900-05-05 12:34:56.1029,1900-12-25 +1900-05-05 12:34:56.103,1900-12-25 +1900-05-05 12:34:56.1031,1900-12-25 +1900-05-05 12:34:56.1032,1900-12-25 +1900-05-05 12:34:56.1033,1900-12-25 +1900-05-05 12:34:56.1034,1900-12-25 +1900-05-05 12:34:56.1035,1900-12-25 +1900-05-05 12:34:56.1036,1900-12-25 +1900-05-05 12:34:56.1037,1900-12-25 +1900-05-05 12:34:56.1038,1900-12-25 +1900-05-05 12:34:56.1039,1900-12-25 +1900-05-05 12:34:56.104,1900-12-25 +1900-05-05 12:34:56.1041,1900-12-25 +1900-05-05 12:34:56.1042,1900-12-25 +1900-05-05 12:34:56.1043,1900-12-25 +1900-05-05 12:34:56.1044,1900-12-25 +1900-05-05 12:34:56.1045,1900-12-25 +1900-05-05 12:34:56.1046,1900-12-25 +1900-05-05 12:34:56.1047,1900-12-25 +1900-05-05 12:34:56.1048,1900-12-25 +1900-05-05 12:34:56.1049,1900-12-25 +1900-05-05 12:34:56.105,1900-12-25 +1900-05-05 12:34:56.1051,1900-12-25 +1900-05-05 12:34:56.1052,1900-12-25 +1900-05-05 12:34:56.1053,1900-12-25 +1900-05-05 12:34:56.1054,1900-12-25 +1900-05-05 12:34:56.1055,1900-12-25 +1900-05-05 12:34:56.1056,1900-12-25 +1900-05-05 12:34:56.1057,1900-12-25 +1900-05-05 12:34:56.1058,1900-12-25 +1900-05-05 12:34:56.1059,1900-12-25 +1900-05-05 12:34:56.106,1900-12-25 +1900-05-05 12:34:56.1061,1900-12-25 +1900-05-05 12:34:56.1062,1900-12-25 +1900-05-05 12:34:56.1063,1900-12-25 +1900-05-05 12:34:56.1064,1900-12-25 +1900-05-05 12:34:56.1065,1900-12-25 +1900-05-05 12:34:56.1066,1900-12-25 +1900-05-05 12:34:56.1067,1900-12-25 +1900-05-05 12:34:56.1068,1900-12-25 +1900-05-05 12:34:56.1069,1900-12-25 +1900-05-05 12:34:56.107,1900-12-25 +1900-05-05 12:34:56.1071,1900-12-25 +1900-05-05 12:34:56.1072,1900-12-25 +1900-05-05 12:34:56.1073,1900-12-25 +1900-05-05 12:34:56.1074,1900-12-25 +1900-05-05 12:34:56.1075,1900-12-25 +1900-05-05 12:34:56.1076,1900-12-25 +1900-05-05 12:34:56.1077,1900-12-25 +1900-05-05 12:34:56.1078,1900-12-25 +1900-05-05 12:34:56.1079,1900-12-25 +1900-05-05 12:34:56.108,1900-12-25 +1900-05-05 12:34:56.1081,1900-12-25 +1900-05-05 12:34:56.1082,1900-12-25 +1900-05-05 12:34:56.1083,1900-12-25 +1900-05-05 12:34:56.1084,1900-12-25 +1900-05-05 12:34:56.1085,1900-12-25 +1900-05-05 12:34:56.1086,1900-12-25 +1900-05-05 12:34:56.1087,1900-12-25 +1900-05-05 12:34:56.1088,1900-12-25 +1900-05-05 12:34:56.1089,1900-12-25 +1900-05-05 12:34:56.109,1900-12-25 +1900-05-05 12:34:56.1091,1900-12-25 +1900-05-05 12:34:56.1092,1900-12-25 +1900-05-05 12:34:56.1093,1900-12-25 +1900-05-05 12:34:56.1094,1900-12-25 +1900-05-05 12:34:56.1095,1900-12-25 +1900-05-05 12:34:56.1096,1900-12-25 +1900-05-05 12:34:56.1097,1900-12-25 +1900-05-05 12:34:56.1098,1900-12-25 +1900-05-05 12:34:56.1099,1900-12-25 +1900-05-05 12:34:56.11,1900-12-25 +1900-05-05 12:34:56.1101,1900-12-25 +1900-05-05 12:34:56.1102,1900-12-25 +1900-05-05 12:34:56.1103,1900-12-25 +1900-05-05 12:34:56.1104,1900-12-25 +1900-05-05 12:34:56.1105,1900-12-25 +1900-05-05 12:34:56.1106,1900-12-25 +1900-05-05 12:34:56.1107,1900-12-25 +1900-05-05 12:34:56.1108,1900-12-25 +1900-05-05 12:34:56.1109,1900-12-25 +1900-05-05 12:34:56.111,1900-12-25 +1900-05-05 12:34:56.1111,1900-12-25 +1900-05-05 12:34:56.1112,1900-12-25 +1900-05-05 12:34:56.1113,1900-12-25 +1900-05-05 12:34:56.1114,1900-12-25 +1900-05-05 12:34:56.1115,1900-12-25 +1900-05-05 12:34:56.1116,1900-12-25 +1900-05-05 12:34:56.1117,1900-12-25 +1900-05-05 12:34:56.1118,1900-12-25 +1900-05-05 12:34:56.1119,1900-12-25 +1900-05-05 12:34:56.112,1900-12-25 +1900-05-05 12:34:56.1121,1900-12-25 +1900-05-05 12:34:56.1122,1900-12-25 +1900-05-05 12:34:56.1123,1900-12-25 +1900-05-05 12:34:56.1124,1900-12-25 +1900-05-05 12:34:56.1125,1900-12-25 +1900-05-05 12:34:56.1126,1900-12-25 +1900-05-05 12:34:56.1127,1900-12-25 +1900-05-05 12:34:56.1128,1900-12-25 +1900-05-05 12:34:56.1129,1900-12-25 +1900-05-05 12:34:56.113,1900-12-25 +1900-05-05 12:34:56.1131,1900-12-25 +1900-05-05 12:34:56.1132,1900-12-25 +1900-05-05 12:34:56.1133,1900-12-25 +1900-05-05 12:34:56.1134,1900-12-25 +1900-05-05 12:34:56.1135,1900-12-25 +1900-05-05 12:34:56.1136,1900-12-25 +1900-05-05 12:34:56.1137,1900-12-25 +1900-05-05 12:34:56.1138,1900-12-25 +1900-05-05 12:34:56.1139,1900-12-25 +1900-05-05 12:34:56.114,1900-12-25 +1900-05-05 12:34:56.1141,1900-12-25 +1900-05-05 12:34:56.1142,1900-12-25 +1900-05-05 12:34:56.1143,1900-12-25 +1900-05-05 12:34:56.1144,1900-12-25 +1900-05-05 12:34:56.1145,1900-12-25 +1900-05-05 12:34:56.1146,1900-12-25 +1900-05-05 12:34:56.1147,1900-12-25 +1900-05-05 12:34:56.1148,1900-12-25 +1900-05-05 12:34:56.1149,1900-12-25 +1900-05-05 12:34:56.115,1900-12-25 +1900-05-05 12:34:56.1151,1900-12-25 +1900-05-05 12:34:56.1152,1900-12-25 +1900-05-05 12:34:56.1153,1900-12-25 +1900-05-05 12:34:56.1154,1900-12-25 +1900-05-05 12:34:56.1155,1900-12-25 +1900-05-05 12:34:56.1156,1900-12-25 +1900-05-05 12:34:56.1157,1900-12-25 +1900-05-05 12:34:56.1158,1900-12-25 +1900-05-05 12:34:56.1159,1900-12-25 +1900-05-05 12:34:56.116,1900-12-25 +1900-05-05 12:34:56.1161,1900-12-25 +1900-05-05 12:34:56.1162,1900-12-25 +1900-05-05 12:34:56.1163,1900-12-25 +1900-05-05 12:34:56.1164,1900-12-25 +1900-05-05 12:34:56.1165,1900-12-25 +1900-05-05 12:34:56.1166,1900-12-25 +1900-05-05 12:34:56.1167,1900-12-25 +1900-05-05 12:34:56.1168,1900-12-25 +1900-05-05 12:34:56.1169,1900-12-25 +1900-05-05 12:34:56.117,1900-12-25 +1900-05-05 12:34:56.1171,1900-12-25 +1900-05-05 12:34:56.1172,1900-12-25 +1900-05-05 12:34:56.1173,1900-12-25 +1900-05-05 12:34:56.1174,1900-12-25 +1900-05-05 12:34:56.1175,1900-12-25 +1900-05-05 12:34:56.1176,1900-12-25 +1900-05-05 12:34:56.1177,1900-12-25 +1900-05-05 12:34:56.1178,1900-12-25 +1900-05-05 12:34:56.1179,1900-12-25 +1900-05-05 12:34:56.118,1900-12-25 +1900-05-05 12:34:56.1181,1900-12-25 +1900-05-05 12:34:56.1182,1900-12-25 +1900-05-05 12:34:56.1183,1900-12-25 +1900-05-05 12:34:56.1184,1900-12-25 +1900-05-05 12:34:56.1185,1900-12-25 +1900-05-05 12:34:56.1186,1900-12-25 +1900-05-05 12:34:56.1187,1900-12-25 +1900-05-05 12:34:56.1188,1900-12-25 +1900-05-05 12:34:56.1189,1900-12-25 +1900-05-05 12:34:56.119,1900-12-25 +1900-05-05 12:34:56.1191,1900-12-25 +1900-05-05 12:34:56.1192,1900-12-25 +1900-05-05 12:34:56.1193,1900-12-25 +1900-05-05 12:34:56.1194,1900-12-25 +1900-05-05 12:34:56.1195,1900-12-25 +1900-05-05 12:34:56.1196,1900-12-25 +1900-05-05 12:34:56.1197,1900-12-25 +1900-05-05 12:34:56.1198,1900-12-25 +1900-05-05 12:34:56.1199,1900-12-25 +1900-05-05 12:34:56.12,1900-12-25 +1900-05-05 12:34:56.1201,1900-12-25 +1900-05-05 12:34:56.1202,1900-12-25 +1900-05-05 12:34:56.1203,1900-12-25 +1900-05-05 12:34:56.1204,1900-12-25 +1900-05-05 12:34:56.1205,1900-12-25 +1900-05-05 12:34:56.1206,1900-12-25 +1900-05-05 12:34:56.1207,1900-12-25 +1900-05-05 12:34:56.1208,1900-12-25 +1900-05-05 12:34:56.1209,1900-12-25 +1900-05-05 12:34:56.121,1900-12-25 +1900-05-05 12:34:56.1211,1900-12-25 +1900-05-05 12:34:56.1212,1900-12-25 +1900-05-05 12:34:56.1213,1900-12-25 +1900-05-05 12:34:56.1214,1900-12-25 +1900-05-05 12:34:56.1215,1900-12-25 +1900-05-05 12:34:56.1216,1900-12-25 +1900-05-05 12:34:56.1217,1900-12-25 +1900-05-05 12:34:56.1218,1900-12-25 +1900-05-05 12:34:56.1219,1900-12-25 +1900-05-05 12:34:56.122,1900-12-25 +1900-05-05 12:34:56.1221,1900-12-25 +1900-05-05 12:34:56.1222,1900-12-25 +1900-05-05 12:34:56.1223,1900-12-25 +1900-05-05 12:34:56.1224,1900-12-25 +1900-05-05 12:34:56.1225,1900-12-25 +1900-05-05 12:34:56.1226,1900-12-25 +1900-05-05 12:34:56.1227,1900-12-25 +1900-05-05 12:34:56.1228,1900-12-25 +1900-05-05 12:34:56.1229,1900-12-25 +1900-05-05 12:34:56.123,1900-12-25 +1900-05-05 12:34:56.1231,1900-12-25 +1900-05-05 12:34:56.1232,1900-12-25 +1900-05-05 12:34:56.1233,1900-12-25 +1900-05-05 12:34:56.1234,1900-12-25 +1900-05-05 12:34:56.1235,1900-12-25 +1900-05-05 12:34:56.1236,1900-12-25 +1900-05-05 12:34:56.1237,1900-12-25 +1900-05-05 12:34:56.1238,1900-12-25 +1900-05-05 12:34:56.1239,1900-12-25 +1900-05-05 12:34:56.124,1900-12-25 +1900-05-05 12:34:56.1241,1900-12-25 +1900-05-05 12:34:56.1242,1900-12-25 +1900-05-05 12:34:56.1243,1900-12-25 +1900-05-05 12:34:56.1244,1900-12-25 +1900-05-05 12:34:56.1245,1900-12-25 +1900-05-05 12:34:56.1246,1900-12-25 +1900-05-05 12:34:56.1247,1900-12-25 +1900-05-05 12:34:56.1248,1900-12-25 +1900-05-05 12:34:56.1249,1900-12-25 +1900-05-05 12:34:56.125,1900-12-25 +1900-05-05 12:34:56.1251,1900-12-25 +1900-05-05 12:34:56.1252,1900-12-25 +1900-05-05 12:34:56.1253,1900-12-25 +1900-05-05 12:34:56.1254,1900-12-25 +1900-05-05 12:34:56.1255,1900-12-25 +1900-05-05 12:34:56.1256,1900-12-25 +1900-05-05 12:34:56.1257,1900-12-25 +1900-05-05 12:34:56.1258,1900-12-25 +1900-05-05 12:34:56.1259,1900-12-25 +1900-05-05 12:34:56.126,1900-12-25 +1900-05-05 12:34:56.1261,1900-12-25 +1900-05-05 12:34:56.1262,1900-12-25 +1900-05-05 12:34:56.1263,1900-12-25 +1900-05-05 12:34:56.1264,1900-12-25 +1900-05-05 12:34:56.1265,1900-12-25 +1900-05-05 12:34:56.1266,1900-12-25 +1900-05-05 12:34:56.1267,1900-12-25 +1900-05-05 12:34:56.1268,1900-12-25 +1900-05-05 12:34:56.1269,1900-12-25 +1900-05-05 12:34:56.127,1900-12-25 +1900-05-05 12:34:56.1271,1900-12-25 +1900-05-05 12:34:56.1272,1900-12-25 +1900-05-05 12:34:56.1273,1900-12-25 +1900-05-05 12:34:56.1274,1900-12-25 +1900-05-05 12:34:56.1275,1900-12-25 +1900-05-05 12:34:56.1276,1900-12-25 +1900-05-05 12:34:56.1277,1900-12-25 +1900-05-05 12:34:56.1278,1900-12-25 +1900-05-05 12:34:56.1279,1900-12-25 +1900-05-05 12:34:56.128,1900-12-25 +1900-05-05 12:34:56.1281,1900-12-25 +1900-05-05 12:34:56.1282,1900-12-25 +1900-05-05 12:34:56.1283,1900-12-25 +1900-05-05 12:34:56.1284,1900-12-25 +1900-05-05 12:34:56.1285,1900-12-25 +1900-05-05 12:34:56.1286,1900-12-25 +1900-05-05 12:34:56.1287,1900-12-25 +1900-05-05 12:34:56.1288,1900-12-25 +1900-05-05 12:34:56.1289,1900-12-25 +1900-05-05 12:34:56.129,1900-12-25 +1900-05-05 12:34:56.1291,1900-12-25 +1900-05-05 12:34:56.1292,1900-12-25 +1900-05-05 12:34:56.1293,1900-12-25 +1900-05-05 12:34:56.1294,1900-12-25 +1900-05-05 12:34:56.1295,1900-12-25 +1900-05-05 12:34:56.1296,1900-12-25 +1900-05-05 12:34:56.1297,1900-12-25 +1900-05-05 12:34:56.1298,1900-12-25 +1900-05-05 12:34:56.1299,1900-12-25 +1900-05-05 12:34:56.13,1900-12-25 +1900-05-05 12:34:56.1301,1900-12-25 +1900-05-05 12:34:56.1302,1900-12-25 +1900-05-05 12:34:56.1303,1900-12-25 +1900-05-05 12:34:56.1304,1900-12-25 +1900-05-05 12:34:56.1305,1900-12-25 +1900-05-05 12:34:56.1306,1900-12-25 +1900-05-05 12:34:56.1307,1900-12-25 +1900-05-05 12:34:56.1308,1900-12-25 +1900-05-05 12:34:56.1309,1900-12-25 +1900-05-05 12:34:56.131,1900-12-25 +1900-05-05 12:34:56.1311,1900-12-25 +1900-05-05 12:34:56.1312,1900-12-25 +1900-05-05 12:34:56.1313,1900-12-25 +1900-05-05 12:34:56.1314,1900-12-25 +1900-05-05 12:34:56.1315,1900-12-25 +1900-05-05 12:34:56.1316,1900-12-25 +1900-05-05 12:34:56.1317,1900-12-25 +1900-05-05 12:34:56.1318,1900-12-25 +1900-05-05 12:34:56.1319,1900-12-25 +1900-05-05 12:34:56.132,1900-12-25 +1900-05-05 12:34:56.1321,1900-12-25 +1900-05-05 12:34:56.1322,1900-12-25 +1900-05-05 12:34:56.1323,1900-12-25 +1900-05-05 12:34:56.1324,1900-12-25 +1900-05-05 12:34:56.1325,1900-12-25 +1900-05-05 12:34:56.1326,1900-12-25 +1900-05-05 12:34:56.1327,1900-12-25 +1900-05-05 12:34:56.1328,1900-12-25 +1900-05-05 12:34:56.1329,1900-12-25 +1900-05-05 12:34:56.133,1900-12-25 +1900-05-05 12:34:56.1331,1900-12-25 +1900-05-05 12:34:56.1332,1900-12-25 +1900-05-05 12:34:56.1333,1900-12-25 +1900-05-05 12:34:56.1334,1900-12-25 +1900-05-05 12:34:56.1335,1900-12-25 +1900-05-05 12:34:56.1336,1900-12-25 +1900-05-05 12:34:56.1337,1900-12-25 +1900-05-05 12:34:56.1338,1900-12-25 +1900-05-05 12:34:56.1339,1900-12-25 +1900-05-05 12:34:56.134,1900-12-25 +1900-05-05 12:34:56.1341,1900-12-25 +1900-05-05 12:34:56.1342,1900-12-25 +1900-05-05 12:34:56.1343,1900-12-25 +1900-05-05 12:34:56.1344,1900-12-25 +1900-05-05 12:34:56.1345,1900-12-25 +1900-05-05 12:34:56.1346,1900-12-25 +1900-05-05 12:34:56.1347,1900-12-25 +1900-05-05 12:34:56.1348,1900-12-25 +1900-05-05 12:34:56.1349,1900-12-25 +1900-05-05 12:34:56.135,1900-12-25 +1900-05-05 12:34:56.1351,1900-12-25 +1900-05-05 12:34:56.1352,1900-12-25 +1900-05-05 12:34:56.1353,1900-12-25 +1900-05-05 12:34:56.1354,1900-12-25 +1900-05-05 12:34:56.1355,1900-12-25 +1900-05-05 12:34:56.1356,1900-12-25 +1900-05-05 12:34:56.1357,1900-12-25 +1900-05-05 12:34:56.1358,1900-12-25 +1900-05-05 12:34:56.1359,1900-12-25 +1900-05-05 12:34:56.136,1900-12-25 +1900-05-05 12:34:56.1361,1900-12-25 +1900-05-05 12:34:56.1362,1900-12-25 +1900-05-05 12:34:56.1363,1900-12-25 +1900-05-05 12:34:56.1364,1900-12-25 +1900-05-05 12:34:56.1365,1900-12-25 +1900-05-05 12:34:56.1366,1900-12-25 +1900-05-05 12:34:56.1367,1900-12-25 +1900-05-05 12:34:56.1368,1900-12-25 +1900-05-05 12:34:56.1369,1900-12-25 +1900-05-05 12:34:56.137,1900-12-25 +1900-05-05 12:34:56.1371,1900-12-25 +1900-05-05 12:34:56.1372,1900-12-25 +1900-05-05 12:34:56.1373,1900-12-25 +1900-05-05 12:34:56.1374,1900-12-25 +1900-05-05 12:34:56.1375,1900-12-25 +1900-05-05 12:34:56.1376,1900-12-25 +1900-05-05 12:34:56.1377,1900-12-25 +1900-05-05 12:34:56.1378,1900-12-25 +1900-05-05 12:34:56.1379,1900-12-25 +1900-05-05 12:34:56.138,1900-12-25 +1900-05-05 12:34:56.1381,1900-12-25 +1900-05-05 12:34:56.1382,1900-12-25 +1900-05-05 12:34:56.1383,1900-12-25 +1900-05-05 12:34:56.1384,1900-12-25 +1900-05-05 12:34:56.1385,1900-12-25 +1900-05-05 12:34:56.1386,1900-12-25 +1900-05-05 12:34:56.1387,1900-12-25 +1900-05-05 12:34:56.1388,1900-12-25 +1900-05-05 12:34:56.1389,1900-12-25 +1900-05-05 12:34:56.139,1900-12-25 +1900-05-05 12:34:56.1391,1900-12-25 +1900-05-05 12:34:56.1392,1900-12-25 +1900-05-05 12:34:56.1393,1900-12-25 +1900-05-05 12:34:56.1394,1900-12-25 +1900-05-05 12:34:56.1395,1900-12-25 +1900-05-05 12:34:56.1396,1900-12-25 +1900-05-05 12:34:56.1397,1900-12-25 +1900-05-05 12:34:56.1398,1900-12-25 +1900-05-05 12:34:56.1399,1900-12-25 +1900-05-05 12:34:56.14,1900-12-25 +1900-05-05 12:34:56.1401,1900-12-25 +1900-05-05 12:34:56.1402,1900-12-25 +1900-05-05 12:34:56.1403,1900-12-25 +1900-05-05 12:34:56.1404,1900-12-25 +1900-05-05 12:34:56.1405,1900-12-25 +1900-05-05 12:34:56.1406,1900-12-25 +1900-05-05 12:34:56.1407,1900-12-25 +1900-05-05 12:34:56.1408,1900-12-25 +1900-05-05 12:34:56.1409,1900-12-25 +1900-05-05 12:34:56.141,1900-12-25 +1900-05-05 12:34:56.1411,1900-12-25 +1900-05-05 12:34:56.1412,1900-12-25 +1900-05-05 12:34:56.1413,1900-12-25 +1900-05-05 12:34:56.1414,1900-12-25 +1900-05-05 12:34:56.1415,1900-12-25 +1900-05-05 12:34:56.1416,1900-12-25 +1900-05-05 12:34:56.1417,1900-12-25 +1900-05-05 12:34:56.1418,1900-12-25 +1900-05-05 12:34:56.1419,1900-12-25 +1900-05-05 12:34:56.142,1900-12-25 +1900-05-05 12:34:56.1421,1900-12-25 +1900-05-05 12:34:56.1422,1900-12-25 +1900-05-05 12:34:56.1423,1900-12-25 +1900-05-05 12:34:56.1424,1900-12-25 +1900-05-05 12:34:56.1425,1900-12-25 +1900-05-05 12:34:56.1426,1900-12-25 +1900-05-05 12:34:56.1427,1900-12-25 +1900-05-05 12:34:56.1428,1900-12-25 +1900-05-05 12:34:56.1429,1900-12-25 +1900-05-05 12:34:56.143,1900-12-25 +1900-05-05 12:34:56.1431,1900-12-25 +1900-05-05 12:34:56.1432,1900-12-25 +1900-05-05 12:34:56.1433,1900-12-25 +1900-05-05 12:34:56.1434,1900-12-25 +1900-05-05 12:34:56.1435,1900-12-25 +1900-05-05 12:34:56.1436,1900-12-25 +1900-05-05 12:34:56.1437,1900-12-25 +1900-05-05 12:34:56.1438,1900-12-25 +1900-05-05 12:34:56.1439,1900-12-25 +1900-05-05 12:34:56.144,1900-12-25 +1900-05-05 12:34:56.1441,1900-12-25 +1900-05-05 12:34:56.1442,1900-12-25 +1900-05-05 12:34:56.1443,1900-12-25 +1900-05-05 12:34:56.1444,1900-12-25 +1900-05-05 12:34:56.1445,1900-12-25 +1900-05-05 12:34:56.1446,1900-12-25 +1900-05-05 12:34:56.1447,1900-12-25 +1900-05-05 12:34:56.1448,1900-12-25 +1900-05-05 12:34:56.1449,1900-12-25 +1900-05-05 12:34:56.145,1900-12-25 +1900-05-05 12:34:56.1451,1900-12-25 +1900-05-05 12:34:56.1452,1900-12-25 +1900-05-05 12:34:56.1453,1900-12-25 +1900-05-05 12:34:56.1454,1900-12-25 +1900-05-05 12:34:56.1455,1900-12-25 +1900-05-05 12:34:56.1456,1900-12-25 +1900-05-05 12:34:56.1457,1900-12-25 +1900-05-05 12:34:56.1458,1900-12-25 +1900-05-05 12:34:56.1459,1900-12-25 +1900-05-05 12:34:56.146,1900-12-25 +1900-05-05 12:34:56.1461,1900-12-25 +1900-05-05 12:34:56.1462,1900-12-25 +1900-05-05 12:34:56.1463,1900-12-25 +1900-05-05 12:34:56.1464,1900-12-25 +1900-05-05 12:34:56.1465,1900-12-25 +1900-05-05 12:34:56.1466,1900-12-25 +1900-05-05 12:34:56.1467,1900-12-25 +1900-05-05 12:34:56.1468,1900-12-25 +1900-05-05 12:34:56.1469,1900-12-25 +1900-05-05 12:34:56.147,1900-12-25 +1900-05-05 12:34:56.1471,1900-12-25 +1900-05-05 12:34:56.1472,1900-12-25 +1900-05-05 12:34:56.1473,1900-12-25 +1900-05-05 12:34:56.1474,1900-12-25 +1900-05-05 12:34:56.1475,1900-12-25 +1900-05-05 12:34:56.1476,1900-12-25 +1900-05-05 12:34:56.1477,1900-12-25 +1900-05-05 12:34:56.1478,1900-12-25 +1900-05-05 12:34:56.1479,1900-12-25 +1900-05-05 12:34:56.148,1900-12-25 +1900-05-05 12:34:56.1481,1900-12-25 +1900-05-05 12:34:56.1482,1900-12-25 +1900-05-05 12:34:56.1483,1900-12-25 +1900-05-05 12:34:56.1484,1900-12-25 +1900-05-05 12:34:56.1485,1900-12-25 +1900-05-05 12:34:56.1486,1900-12-25 +1900-05-05 12:34:56.1487,1900-12-25 +1900-05-05 12:34:56.1488,1900-12-25 +1900-05-05 12:34:56.1489,1900-12-25 +1900-05-05 12:34:56.149,1900-12-25 +1900-05-05 12:34:56.1491,1900-12-25 +1900-05-05 12:34:56.1492,1900-12-25 +1900-05-05 12:34:56.1493,1900-12-25 +1900-05-05 12:34:56.1494,1900-12-25 +1900-05-05 12:34:56.1495,1900-12-25 +1900-05-05 12:34:56.1496,1900-12-25 +1900-05-05 12:34:56.1497,1900-12-25 +1900-05-05 12:34:56.1498,1900-12-25 +1900-05-05 12:34:56.1499,1900-12-25 +1900-05-05 12:34:56.15,1900-12-25 +1900-05-05 12:34:56.1501,1900-12-25 +1900-05-05 12:34:56.1502,1900-12-25 +1900-05-05 12:34:56.1503,1900-12-25 +1900-05-05 12:34:56.1504,1900-12-25 +1900-05-05 12:34:56.1505,1900-12-25 +1900-05-05 12:34:56.1506,1900-12-25 +1900-05-05 12:34:56.1507,1900-12-25 +1900-05-05 12:34:56.1508,1900-12-25 +1900-05-05 12:34:56.1509,1900-12-25 +1900-05-05 12:34:56.151,1900-12-25 +1900-05-05 12:34:56.1511,1900-12-25 +1900-05-05 12:34:56.1512,1900-12-25 +1900-05-05 12:34:56.1513,1900-12-25 +1900-05-05 12:34:56.1514,1900-12-25 +1900-05-05 12:34:56.1515,1900-12-25 +1900-05-05 12:34:56.1516,1900-12-25 +1900-05-05 12:34:56.1517,1900-12-25 +1900-05-05 12:34:56.1518,1900-12-25 +1900-05-05 12:34:56.1519,1900-12-25 +1900-05-05 12:34:56.152,1900-12-25 +1900-05-05 12:34:56.1521,1900-12-25 +1900-05-05 12:34:56.1522,1900-12-25 +1900-05-05 12:34:56.1523,1900-12-25 +1900-05-05 12:34:56.1524,1900-12-25 +1900-05-05 12:34:56.1525,1900-12-25 +1900-05-05 12:34:56.1526,1900-12-25 +1900-05-05 12:34:56.1527,1900-12-25 +1900-05-05 12:34:56.1528,1900-12-25 +1900-05-05 12:34:56.1529,1900-12-25 +1900-05-05 12:34:56.153,1900-12-25 +1900-05-05 12:34:56.1531,1900-12-25 +1900-05-05 12:34:56.1532,1900-12-25 +1900-05-05 12:34:56.1533,1900-12-25 +1900-05-05 12:34:56.1534,1900-12-25 +1900-05-05 12:34:56.1535,1900-12-25 +1900-05-05 12:34:56.1536,1900-12-25 +1900-05-05 12:34:56.1537,1900-12-25 +1900-05-05 12:34:56.1538,1900-12-25 +1900-05-05 12:34:56.1539,1900-12-25 +1900-05-05 12:34:56.154,1900-12-25 +1900-05-05 12:34:56.1541,1900-12-25 +1900-05-05 12:34:56.1542,1900-12-25 +1900-05-05 12:34:56.1543,1900-12-25 +1900-05-05 12:34:56.1544,1900-12-25 +1900-05-05 12:34:56.1545,1900-12-25 +1900-05-05 12:34:56.1546,1900-12-25 +1900-05-05 12:34:56.1547,1900-12-25 +1900-05-05 12:34:56.1548,1900-12-25 +1900-05-05 12:34:56.1549,1900-12-25 +1900-05-05 12:34:56.155,1900-12-25 +1900-05-05 12:34:56.1551,1900-12-25 +1900-05-05 12:34:56.1552,1900-12-25 +1900-05-05 12:34:56.1553,1900-12-25 +1900-05-05 12:34:56.1554,1900-12-25 +1900-05-05 12:34:56.1555,1900-12-25 +1900-05-05 12:34:56.1556,1900-12-25 +1900-05-05 12:34:56.1557,1900-12-25 +1900-05-05 12:34:56.1558,1900-12-25 +1900-05-05 12:34:56.1559,1900-12-25 +1900-05-05 12:34:56.156,1900-12-25 +1900-05-05 12:34:56.1561,1900-12-25 +1900-05-05 12:34:56.1562,1900-12-25 +1900-05-05 12:34:56.1563,1900-12-25 +1900-05-05 12:34:56.1564,1900-12-25 +1900-05-05 12:34:56.1565,1900-12-25 +1900-05-05 12:34:56.1566,1900-12-25 +1900-05-05 12:34:56.1567,1900-12-25 +1900-05-05 12:34:56.1568,1900-12-25 +1900-05-05 12:34:56.1569,1900-12-25 +1900-05-05 12:34:56.157,1900-12-25 +1900-05-05 12:34:56.1571,1900-12-25 +1900-05-05 12:34:56.1572,1900-12-25 +1900-05-05 12:34:56.1573,1900-12-25 +1900-05-05 12:34:56.1574,1900-12-25 +1900-05-05 12:34:56.1575,1900-12-25 +1900-05-05 12:34:56.1576,1900-12-25 +1900-05-05 12:34:56.1577,1900-12-25 +1900-05-05 12:34:56.1578,1900-12-25 +1900-05-05 12:34:56.1579,1900-12-25 +1900-05-05 12:34:56.158,1900-12-25 +1900-05-05 12:34:56.1581,1900-12-25 +1900-05-05 12:34:56.1582,1900-12-25 +1900-05-05 12:34:56.1583,1900-12-25 +1900-05-05 12:34:56.1584,1900-12-25 +1900-05-05 12:34:56.1585,1900-12-25 +1900-05-05 12:34:56.1586,1900-12-25 +1900-05-05 12:34:56.1587,1900-12-25 +1900-05-05 12:34:56.1588,1900-12-25 +1900-05-05 12:34:56.1589,1900-12-25 +1900-05-05 12:34:56.159,1900-12-25 +1900-05-05 12:34:56.1591,1900-12-25 +1900-05-05 12:34:56.1592,1900-12-25 +1900-05-05 12:34:56.1593,1900-12-25 +1900-05-05 12:34:56.1594,1900-12-25 +1900-05-05 12:34:56.1595,1900-12-25 +1900-05-05 12:34:56.1596,1900-12-25 +1900-05-05 12:34:56.1597,1900-12-25 +1900-05-05 12:34:56.1598,1900-12-25 +1900-05-05 12:34:56.1599,1900-12-25 +1900-05-05 12:34:56.16,1900-12-25 +1900-05-05 12:34:56.1601,1900-12-25 +1900-05-05 12:34:56.1602,1900-12-25 +1900-05-05 12:34:56.1603,1900-12-25 +1900-05-05 12:34:56.1604,1900-12-25 +1900-05-05 12:34:56.1605,1900-12-25 +1900-05-05 12:34:56.1606,1900-12-25 +1900-05-05 12:34:56.1607,1900-12-25 +1900-05-05 12:34:56.1608,1900-12-25 +1900-05-05 12:34:56.1609,1900-12-25 +1900-05-05 12:34:56.161,1900-12-25 +1900-05-05 12:34:56.1611,1900-12-25 +1900-05-05 12:34:56.1612,1900-12-25 +1900-05-05 12:34:56.1613,1900-12-25 +1900-05-05 12:34:56.1614,1900-12-25 +1900-05-05 12:34:56.1615,1900-12-25 +1900-05-05 12:34:56.1616,1900-12-25 +1900-05-05 12:34:56.1617,1900-12-25 +1900-05-05 12:34:56.1618,1900-12-25 +1900-05-05 12:34:56.1619,1900-12-25 +1900-05-05 12:34:56.162,1900-12-25 +1900-05-05 12:34:56.1621,1900-12-25 +1900-05-05 12:34:56.1622,1900-12-25 +1900-05-05 12:34:56.1623,1900-12-25 +1900-05-05 12:34:56.1624,1900-12-25 +1900-05-05 12:34:56.1625,1900-12-25 +1900-05-05 12:34:56.1626,1900-12-25 +1900-05-05 12:34:56.1627,1900-12-25 +1900-05-05 12:34:56.1628,1900-12-25 +1900-05-05 12:34:56.1629,1900-12-25 +1900-05-05 12:34:56.163,1900-12-25 +1900-05-05 12:34:56.1631,1900-12-25 +1900-05-05 12:34:56.1632,1900-12-25 +1900-05-05 12:34:56.1633,1900-12-25 +1900-05-05 12:34:56.1634,1900-12-25 +1900-05-05 12:34:56.1635,1900-12-25 +1900-05-05 12:34:56.1636,1900-12-25 +1900-05-05 12:34:56.1637,1900-12-25 +1900-05-05 12:34:56.1638,1900-12-25 +1900-05-05 12:34:56.1639,1900-12-25 +1900-05-05 12:34:56.164,1900-12-25 +1900-05-05 12:34:56.1641,1900-12-25 +1900-05-05 12:34:56.1642,1900-12-25 +1900-05-05 12:34:56.1643,1900-12-25 +1900-05-05 12:34:56.1644,1900-12-25 +1900-05-05 12:34:56.1645,1900-12-25 +1900-05-05 12:34:56.1646,1900-12-25 +1900-05-05 12:34:56.1647,1900-12-25 +1900-05-05 12:34:56.1648,1900-12-25 +1900-05-05 12:34:56.1649,1900-12-25 +1900-05-05 12:34:56.165,1900-12-25 +1900-05-05 12:34:56.1651,1900-12-25 +1900-05-05 12:34:56.1652,1900-12-25 +1900-05-05 12:34:56.1653,1900-12-25 +1900-05-05 12:34:56.1654,1900-12-25 +1900-05-05 12:34:56.1655,1900-12-25 +1900-05-05 12:34:56.1656,1900-12-25 +1900-05-05 12:34:56.1657,1900-12-25 +1900-05-05 12:34:56.1658,1900-12-25 +1900-05-05 12:34:56.1659,1900-12-25 +1900-05-05 12:34:56.166,1900-12-25 +1900-05-05 12:34:56.1661,1900-12-25 +1900-05-05 12:34:56.1662,1900-12-25 +1900-05-05 12:34:56.1663,1900-12-25 +1900-05-05 12:34:56.1664,1900-12-25 +1900-05-05 12:34:56.1665,1900-12-25 +1900-05-05 12:34:56.1666,1900-12-25 +1900-05-05 12:34:56.1667,1900-12-25 +1900-05-05 12:34:56.1668,1900-12-25 +1900-05-05 12:34:56.1669,1900-12-25 +1900-05-05 12:34:56.167,1900-12-25 +1900-05-05 12:34:56.1671,1900-12-25 +1900-05-05 12:34:56.1672,1900-12-25 +1900-05-05 12:34:56.1673,1900-12-25 +1900-05-05 12:34:56.1674,1900-12-25 +1900-05-05 12:34:56.1675,1900-12-25 +1900-05-05 12:34:56.1676,1900-12-25 +1900-05-05 12:34:56.1677,1900-12-25 +1900-05-05 12:34:56.1678,1900-12-25 +1900-05-05 12:34:56.1679,1900-12-25 +1900-05-05 12:34:56.168,1900-12-25 +1900-05-05 12:34:56.1681,1900-12-25 +1900-05-05 12:34:56.1682,1900-12-25 +1900-05-05 12:34:56.1683,1900-12-25 +1900-05-05 12:34:56.1684,1900-12-25 +1900-05-05 12:34:56.1685,1900-12-25 +1900-05-05 12:34:56.1686,1900-12-25 +1900-05-05 12:34:56.1687,1900-12-25 +1900-05-05 12:34:56.1688,1900-12-25 +1900-05-05 12:34:56.1689,1900-12-25 +1900-05-05 12:34:56.169,1900-12-25 +1900-05-05 12:34:56.1691,1900-12-25 +1900-05-05 12:34:56.1692,1900-12-25 +1900-05-05 12:34:56.1693,1900-12-25 +1900-05-05 12:34:56.1694,1900-12-25 +1900-05-05 12:34:56.1695,1900-12-25 +1900-05-05 12:34:56.1696,1900-12-25 +1900-05-05 12:34:56.1697,1900-12-25 +1900-05-05 12:34:56.1698,1900-12-25 +1900-05-05 12:34:56.1699,1900-12-25 +1900-05-05 12:34:56.17,1900-12-25 +1900-05-05 12:34:56.1701,1900-12-25 +1900-05-05 12:34:56.1702,1900-12-25 +1900-05-05 12:34:56.1703,1900-12-25 +1900-05-05 12:34:56.1704,1900-12-25 +1900-05-05 12:34:56.1705,1900-12-25 +1900-05-05 12:34:56.1706,1900-12-25 +1900-05-05 12:34:56.1707,1900-12-25 +1900-05-05 12:34:56.1708,1900-12-25 +1900-05-05 12:34:56.1709,1900-12-25 +1900-05-05 12:34:56.171,1900-12-25 +1900-05-05 12:34:56.1711,1900-12-25 +1900-05-05 12:34:56.1712,1900-12-25 +1900-05-05 12:34:56.1713,1900-12-25 +1900-05-05 12:34:56.1714,1900-12-25 +1900-05-05 12:34:56.1715,1900-12-25 +1900-05-05 12:34:56.1716,1900-12-25 +1900-05-05 12:34:56.1717,1900-12-25 +1900-05-05 12:34:56.1718,1900-12-25 +1900-05-05 12:34:56.1719,1900-12-25 +1900-05-05 12:34:56.172,1900-12-25 +1900-05-05 12:34:56.1721,1900-12-25 +1900-05-05 12:34:56.1722,1900-12-25 +1900-05-05 12:34:56.1723,1900-12-25 +1900-05-05 12:34:56.1724,1900-12-25 +1900-05-05 12:34:56.1725,1900-12-25 +1900-05-05 12:34:56.1726,1900-12-25 +1900-05-05 12:34:56.1727,1900-12-25 +1900-05-05 12:34:56.1728,1900-12-25 +1900-05-05 12:34:56.1729,1900-12-25 +1900-05-05 12:34:56.173,1900-12-25 +1900-05-05 12:34:56.1731,1900-12-25 +1900-05-05 12:34:56.1732,1900-12-25 +1900-05-05 12:34:56.1733,1900-12-25 +1900-05-05 12:34:56.1734,1900-12-25 +1900-05-05 12:34:56.1735,1900-12-25 +1900-05-05 12:34:56.1736,1900-12-25 +1900-05-05 12:34:56.1737,1900-12-25 +1900-05-05 12:34:56.1738,1900-12-25 +1900-05-05 12:34:56.1739,1900-12-25 +1900-05-05 12:34:56.174,1900-12-25 +1900-05-05 12:34:56.1741,1900-12-25 +1900-05-05 12:34:56.1742,1900-12-25 +1900-05-05 12:34:56.1743,1900-12-25 +1900-05-05 12:34:56.1744,1900-12-25 +1900-05-05 12:34:56.1745,1900-12-25 +1900-05-05 12:34:56.1746,1900-12-25 +1900-05-05 12:34:56.1747,1900-12-25 +1900-05-05 12:34:56.1748,1900-12-25 +1900-05-05 12:34:56.1749,1900-12-25 +1900-05-05 12:34:56.175,1900-12-25 +1900-05-05 12:34:56.1751,1900-12-25 +1900-05-05 12:34:56.1752,1900-12-25 +1900-05-05 12:34:56.1753,1900-12-25 +1900-05-05 12:34:56.1754,1900-12-25 +1900-05-05 12:34:56.1755,1900-12-25 +1900-05-05 12:34:56.1756,1900-12-25 +1900-05-05 12:34:56.1757,1900-12-25 +1900-05-05 12:34:56.1758,1900-12-25 +1900-05-05 12:34:56.1759,1900-12-25 +1900-05-05 12:34:56.176,1900-12-25 +1900-05-05 12:34:56.1761,1900-12-25 +1900-05-05 12:34:56.1762,1900-12-25 +1900-05-05 12:34:56.1763,1900-12-25 +1900-05-05 12:34:56.1764,1900-12-25 +1900-05-05 12:34:56.1765,1900-12-25 +1900-05-05 12:34:56.1766,1900-12-25 +1900-05-05 12:34:56.1767,1900-12-25 +1900-05-05 12:34:56.1768,1900-12-25 +1900-05-05 12:34:56.1769,1900-12-25 +1900-05-05 12:34:56.177,1900-12-25 +1900-05-05 12:34:56.1771,1900-12-25 +1900-05-05 12:34:56.1772,1900-12-25 +1900-05-05 12:34:56.1773,1900-12-25 +1900-05-05 12:34:56.1774,1900-12-25 +1900-05-05 12:34:56.1775,1900-12-25 +1900-05-05 12:34:56.1776,1900-12-25 +1900-05-05 12:34:56.1777,1900-12-25 +1900-05-05 12:34:56.1778,1900-12-25 +1900-05-05 12:34:56.1779,1900-12-25 +1900-05-05 12:34:56.178,1900-12-25 +1900-05-05 12:34:56.1781,1900-12-25 +1900-05-05 12:34:56.1782,1900-12-25 +1900-05-05 12:34:56.1783,1900-12-25 +1900-05-05 12:34:56.1784,1900-12-25 +1900-05-05 12:34:56.1785,1900-12-25 +1900-05-05 12:34:56.1786,1900-12-25 +1900-05-05 12:34:56.1787,1900-12-25 +1900-05-05 12:34:56.1788,1900-12-25 +1900-05-05 12:34:56.1789,1900-12-25 +1900-05-05 12:34:56.179,1900-12-25 +1900-05-05 12:34:56.1791,1900-12-25 +1900-05-05 12:34:56.1792,1900-12-25 +1900-05-05 12:34:56.1793,1900-12-25 +1900-05-05 12:34:56.1794,1900-12-25 +1900-05-05 12:34:56.1795,1900-12-25 +1900-05-05 12:34:56.1796,1900-12-25 +1900-05-05 12:34:56.1797,1900-12-25 +1900-05-05 12:34:56.1798,1900-12-25 +1900-05-05 12:34:56.1799,1900-12-25 +1900-05-05 12:34:56.18,1900-12-25 +1900-05-05 12:34:56.1801,1900-12-25 +1900-05-05 12:34:56.1802,1900-12-25 +1900-05-05 12:34:56.1803,1900-12-25 +1900-05-05 12:34:56.1804,1900-12-25 +1900-05-05 12:34:56.1805,1900-12-25 +1900-05-05 12:34:56.1806,1900-12-25 +1900-05-05 12:34:56.1807,1900-12-25 +1900-05-05 12:34:56.1808,1900-12-25 +1900-05-05 12:34:56.1809,1900-12-25 +1900-05-05 12:34:56.181,1900-12-25 +1900-05-05 12:34:56.1811,1900-12-25 +1900-05-05 12:34:56.1812,1900-12-25 +1900-05-05 12:34:56.1813,1900-12-25 +1900-05-05 12:34:56.1814,1900-12-25 +1900-05-05 12:34:56.1815,1900-12-25 +1900-05-05 12:34:56.1816,1900-12-25 +1900-05-05 12:34:56.1817,1900-12-25 +1900-05-05 12:34:56.1818,1900-12-25 +1900-05-05 12:34:56.1819,1900-12-25 +1900-05-05 12:34:56.182,1900-12-25 +1900-05-05 12:34:56.1821,1900-12-25 +1900-05-05 12:34:56.1822,1900-12-25 +1900-05-05 12:34:56.1823,1900-12-25 +1900-05-05 12:34:56.1824,1900-12-25 +1900-05-05 12:34:56.1825,1900-12-25 +1900-05-05 12:34:56.1826,1900-12-25 +1900-05-05 12:34:56.1827,1900-12-25 +1900-05-05 12:34:56.1828,1900-12-25 +1900-05-05 12:34:56.1829,1900-12-25 +1900-05-05 12:34:56.183,1900-12-25 +1900-05-05 12:34:56.1831,1900-12-25 +1900-05-05 12:34:56.1832,1900-12-25 +1900-05-05 12:34:56.1833,1900-12-25 +1900-05-05 12:34:56.1834,1900-12-25 +1900-05-05 12:34:56.1835,1900-12-25 +1900-05-05 12:34:56.1836,1900-12-25 +1900-05-05 12:34:56.1837,1900-12-25 +1900-05-05 12:34:56.1838,1900-12-25 +1900-05-05 12:34:56.1839,1900-12-25 +1900-05-05 12:34:56.184,1900-12-25 +1900-05-05 12:34:56.1841,1900-12-25 +1900-05-05 12:34:56.1842,1900-12-25 +1900-05-05 12:34:56.1843,1900-12-25 +1900-05-05 12:34:56.1844,1900-12-25 +1900-05-05 12:34:56.1845,1900-12-25 +1900-05-05 12:34:56.1846,1900-12-25 +1900-05-05 12:34:56.1847,1900-12-25 +1900-05-05 12:34:56.1848,1900-12-25 +1900-05-05 12:34:56.1849,1900-12-25 +1900-05-05 12:34:56.185,1900-12-25 +1900-05-05 12:34:56.1851,1900-12-25 +1900-05-05 12:34:56.1852,1900-12-25 +1900-05-05 12:34:56.1853,1900-12-25 +1900-05-05 12:34:56.1854,1900-12-25 +1900-05-05 12:34:56.1855,1900-12-25 +1900-05-05 12:34:56.1856,1900-12-25 +1900-05-05 12:34:56.1857,1900-12-25 +1900-05-05 12:34:56.1858,1900-12-25 +1900-05-05 12:34:56.1859,1900-12-25 +1900-05-05 12:34:56.186,1900-12-25 +1900-05-05 12:34:56.1861,1900-12-25 +1900-05-05 12:34:56.1862,1900-12-25 +1900-05-05 12:34:56.1863,1900-12-25 +1900-05-05 12:34:56.1864,1900-12-25 +1900-05-05 12:34:56.1865,1900-12-25 +1900-05-05 12:34:56.1866,1900-12-25 +1900-05-05 12:34:56.1867,1900-12-25 +1900-05-05 12:34:56.1868,1900-12-25 +1900-05-05 12:34:56.1869,1900-12-25 +1900-05-05 12:34:56.187,1900-12-25 +1900-05-05 12:34:56.1871,1900-12-25 +1900-05-05 12:34:56.1872,1900-12-25 +1900-05-05 12:34:56.1873,1900-12-25 +1900-05-05 12:34:56.1874,1900-12-25 +1900-05-05 12:34:56.1875,1900-12-25 +1900-05-05 12:34:56.1876,1900-12-25 +1900-05-05 12:34:56.1877,1900-12-25 +1900-05-05 12:34:56.1878,1900-12-25 +1900-05-05 12:34:56.1879,1900-12-25 +1900-05-05 12:34:56.188,1900-12-25 +1900-05-05 12:34:56.1881,1900-12-25 +1900-05-05 12:34:56.1882,1900-12-25 +1900-05-05 12:34:56.1883,1900-12-25 +1900-05-05 12:34:56.1884,1900-12-25 +1900-05-05 12:34:56.1885,1900-12-25 +1900-05-05 12:34:56.1886,1900-12-25 +1900-05-05 12:34:56.1887,1900-12-25 +1900-05-05 12:34:56.1888,1900-12-25 +1900-05-05 12:34:56.1889,1900-12-25 +1900-05-05 12:34:56.189,1900-12-25 +1900-05-05 12:34:56.1891,1900-12-25 +1900-05-05 12:34:56.1892,1900-12-25 +1900-05-05 12:34:56.1893,1900-12-25 +1900-05-05 12:34:56.1894,1900-12-25 +1900-05-05 12:34:56.1895,1900-12-25 +1900-05-05 12:34:56.1896,1900-12-25 +1900-05-05 12:34:56.1897,1900-12-25 +1900-05-05 12:34:56.1898,1900-12-25 +1900-05-05 12:34:56.1899,1900-12-25 +1900-05-05 12:34:56.19,1900-12-25 +1900-05-05 12:34:56.1901,1900-12-25 +1900-05-05 12:34:56.1902,1900-12-25 +1900-05-05 12:34:56.1903,1900-12-25 +1900-05-05 12:34:56.1904,1900-12-25 +1900-05-05 12:34:56.1905,1900-12-25 +1900-05-05 12:34:56.1906,1900-12-25 +1900-05-05 12:34:56.1907,1900-12-25 +1900-05-05 12:34:56.1908,1900-12-25 +1900-05-05 12:34:56.1909,1900-12-25 +1900-05-05 12:34:56.191,1900-12-25 +1900-05-05 12:34:56.1911,1900-12-25 +1900-05-05 12:34:56.1912,1900-12-25 +1900-05-05 12:34:56.1913,1900-12-25 +1900-05-05 12:34:56.1914,1900-12-25 +1900-05-05 12:34:56.1915,1900-12-25 +1900-05-05 12:34:56.1916,1900-12-25 +1900-05-05 12:34:56.1917,1900-12-25 +1900-05-05 12:34:56.1918,1900-12-25 +1900-05-05 12:34:56.1919,1900-12-25 +1900-05-05 12:34:56.192,1900-12-25 +1900-05-05 12:34:56.1921,1900-12-25 +1900-05-05 12:34:56.1922,1900-12-25 +1900-05-05 12:34:56.1923,1900-12-25 +1900-05-05 12:34:56.1924,1900-12-25 +1900-05-05 12:34:56.1925,1900-12-25 +1900-05-05 12:34:56.1926,1900-12-25 +1900-05-05 12:34:56.1927,1900-12-25 +1900-05-05 12:34:56.1928,1900-12-25 +1900-05-05 12:34:56.1929,1900-12-25 +1900-05-05 12:34:56.193,1900-12-25 +1900-05-05 12:34:56.1931,1900-12-25 +1900-05-05 12:34:56.1932,1900-12-25 +1900-05-05 12:34:56.1933,1900-12-25 +1900-05-05 12:34:56.1934,1900-12-25 +1900-05-05 12:34:56.1935,1900-12-25 +1900-05-05 12:34:56.1936,1900-12-25 +1900-05-05 12:34:56.1937,1900-12-25 +1900-05-05 12:34:56.1938,1900-12-25 +1900-05-05 12:34:56.1939,1900-12-25 +1900-05-05 12:34:56.194,1900-12-25 +1900-05-05 12:34:56.1941,1900-12-25 +1900-05-05 12:34:56.1942,1900-12-25 +1900-05-05 12:34:56.1943,1900-12-25 +1900-05-05 12:34:56.1944,1900-12-25 +1900-05-05 12:34:56.1945,1900-12-25 +1900-05-05 12:34:56.1946,1900-12-25 +1900-05-05 12:34:56.1947,1900-12-25 +1900-05-05 12:34:56.1948,1900-12-25 +1900-05-05 12:34:56.1949,1900-12-25 +1900-05-05 12:34:56.195,1900-12-25 +1900-05-05 12:34:56.1951,1900-12-25 +1900-05-05 12:34:56.1952,1900-12-25 +1900-05-05 12:34:56.1953,1900-12-25 +1900-05-05 12:34:56.1954,1900-12-25 +1900-05-05 12:34:56.1955,1900-12-25 +1900-05-05 12:34:56.1956,1900-12-25 +1900-05-05 12:34:56.1957,1900-12-25 +1900-05-05 12:34:56.1958,1900-12-25 +1900-05-05 12:34:56.1959,1900-12-25 +1900-05-05 12:34:56.196,1900-12-25 +1900-05-05 12:34:56.1961,1900-12-25 +1900-05-05 12:34:56.1962,1900-12-25 +1900-05-05 12:34:56.1963,1900-12-25 +1900-05-05 12:34:56.1964,1900-12-25 +1900-05-05 12:34:56.1965,1900-12-25 +1900-05-05 12:34:56.1966,1900-12-25 +1900-05-05 12:34:56.1967,1900-12-25 +1900-05-05 12:34:56.1968,1900-12-25 +1900-05-05 12:34:56.1969,1900-12-25 +1900-05-05 12:34:56.197,1900-12-25 +1900-05-05 12:34:56.1971,1900-12-25 +1900-05-05 12:34:56.1972,1900-12-25 +1900-05-05 12:34:56.1973,1900-12-25 +1900-05-05 12:34:56.1974,1900-12-25 +1900-05-05 12:34:56.1975,1900-12-25 +1900-05-05 12:34:56.1976,1900-12-25 +1900-05-05 12:34:56.1977,1900-12-25 +1900-05-05 12:34:56.1978,1900-12-25 +1900-05-05 12:34:56.1979,1900-12-25 +1900-05-05 12:34:56.198,1900-12-25 +1900-05-05 12:34:56.1981,1900-12-25 +1900-05-05 12:34:56.1982,1900-12-25 +1900-05-05 12:34:56.1983,1900-12-25 +1900-05-05 12:34:56.1984,1900-12-25 +1900-05-05 12:34:56.1985,1900-12-25 +1900-05-05 12:34:56.1986,1900-12-25 +1900-05-05 12:34:56.1987,1900-12-25 +1900-05-05 12:34:56.1988,1900-12-25 +1900-05-05 12:34:56.1989,1900-12-25 +1900-05-05 12:34:56.199,1900-12-25 +1900-05-05 12:34:56.1991,1900-12-25 +1900-05-05 12:34:56.1992,1900-12-25 +1900-05-05 12:34:56.1993,1900-12-25 +1900-05-05 12:34:56.1994,1900-12-25 +1900-05-05 12:34:56.1995,1900-12-25 +1900-05-05 12:34:56.1996,1900-12-25 +1900-05-05 12:34:56.1997,1900-12-25 +1900-05-05 12:34:56.1998,1900-12-25 +1900-05-05 12:34:56.1999,1900-12-25 +1901-05-05 12:34:56.1,1901-12-25 +1901-05-05 12:34:56.1001,1901-12-25 +1901-05-05 12:34:56.1002,1901-12-25 +1901-05-05 12:34:56.1003,1901-12-25 +1901-05-05 12:34:56.1004,1901-12-25 +1901-05-05 12:34:56.1005,1901-12-25 +1901-05-05 12:34:56.1006,1901-12-25 +1901-05-05 12:34:56.1007,1901-12-25 +1901-05-05 12:34:56.1008,1901-12-25 +1901-05-05 12:34:56.1009,1901-12-25 +1901-05-05 12:34:56.101,1901-12-25 +1901-05-05 12:34:56.1011,1901-12-25 +1901-05-05 12:34:56.1012,1901-12-25 +1901-05-05 12:34:56.1013,1901-12-25 +1901-05-05 12:34:56.1014,1901-12-25 +1901-05-05 12:34:56.1015,1901-12-25 +1901-05-05 12:34:56.1016,1901-12-25 +1901-05-05 12:34:56.1017,1901-12-25 +1901-05-05 12:34:56.1018,1901-12-25 +1901-05-05 12:34:56.1019,1901-12-25 +1901-05-05 12:34:56.102,1901-12-25 +1901-05-05 12:34:56.1021,1901-12-25 +1901-05-05 12:34:56.1022,1901-12-25 +1901-05-05 12:34:56.1023,1901-12-25 +1901-05-05 12:34:56.1024,1901-12-25 +1901-05-05 12:34:56.1025,1901-12-25 +1901-05-05 12:34:56.1026,1901-12-25 +1901-05-05 12:34:56.1027,1901-12-25 +1901-05-05 12:34:56.1028,1901-12-25 +1901-05-05 12:34:56.1029,1901-12-25 +1901-05-05 12:34:56.103,1901-12-25 +1901-05-05 12:34:56.1031,1901-12-25 +1901-05-05 12:34:56.1032,1901-12-25 +1901-05-05 12:34:56.1033,1901-12-25 +1901-05-05 12:34:56.1034,1901-12-25 +1901-05-05 12:34:56.1035,1901-12-25 +1901-05-05 12:34:56.1036,1901-12-25 +1901-05-05 12:34:56.1037,1901-12-25 +1901-05-05 12:34:56.1038,1901-12-25 +1901-05-05 12:34:56.1039,1901-12-25 +1901-05-05 12:34:56.104,1901-12-25 +1901-05-05 12:34:56.1041,1901-12-25 +1901-05-05 12:34:56.1042,1901-12-25 +1901-05-05 12:34:56.1043,1901-12-25 +1901-05-05 12:34:56.1044,1901-12-25 +1901-05-05 12:34:56.1045,1901-12-25 +1901-05-05 12:34:56.1046,1901-12-25 +1901-05-05 12:34:56.1047,1901-12-25 +1901-05-05 12:34:56.1048,1901-12-25 +1901-05-05 12:34:56.1049,1901-12-25 +1901-05-05 12:34:56.105,1901-12-25 +1901-05-05 12:34:56.1051,1901-12-25 +1901-05-05 12:34:56.1052,1901-12-25 +1901-05-05 12:34:56.1053,1901-12-25 +1901-05-05 12:34:56.1054,1901-12-25 +1901-05-05 12:34:56.1055,1901-12-25 +1901-05-05 12:34:56.1056,1901-12-25 +1901-05-05 12:34:56.1057,1901-12-25 +1901-05-05 12:34:56.1058,1901-12-25 +1901-05-05 12:34:56.1059,1901-12-25 +1901-05-05 12:34:56.106,1901-12-25 +1901-05-05 12:34:56.1061,1901-12-25 +1901-05-05 12:34:56.1062,1901-12-25 +1901-05-05 12:34:56.1063,1901-12-25 +1901-05-05 12:34:56.1064,1901-12-25 +1901-05-05 12:34:56.1065,1901-12-25 +1901-05-05 12:34:56.1066,1901-12-25 +1901-05-05 12:34:56.1067,1901-12-25 +1901-05-05 12:34:56.1068,1901-12-25 +1901-05-05 12:34:56.1069,1901-12-25 +1901-05-05 12:34:56.107,1901-12-25 +1901-05-05 12:34:56.1071,1901-12-25 +1901-05-05 12:34:56.1072,1901-12-25 +1901-05-05 12:34:56.1073,1901-12-25 +1901-05-05 12:34:56.1074,1901-12-25 +1901-05-05 12:34:56.1075,1901-12-25 +1901-05-05 12:34:56.1076,1901-12-25 +1901-05-05 12:34:56.1077,1901-12-25 +1901-05-05 12:34:56.1078,1901-12-25 +1901-05-05 12:34:56.1079,1901-12-25 +1901-05-05 12:34:56.108,1901-12-25 +1901-05-05 12:34:56.1081,1901-12-25 +1901-05-05 12:34:56.1082,1901-12-25 +1901-05-05 12:34:56.1083,1901-12-25 +1901-05-05 12:34:56.1084,1901-12-25 +1901-05-05 12:34:56.1085,1901-12-25 +1901-05-05 12:34:56.1086,1901-12-25 +1901-05-05 12:34:56.1087,1901-12-25 +1901-05-05 12:34:56.1088,1901-12-25 +1901-05-05 12:34:56.1089,1901-12-25 +1901-05-05 12:34:56.109,1901-12-25 +1901-05-05 12:34:56.1091,1901-12-25 +1901-05-05 12:34:56.1092,1901-12-25 +1901-05-05 12:34:56.1093,1901-12-25 +1901-05-05 12:34:56.1094,1901-12-25 +1901-05-05 12:34:56.1095,1901-12-25 +1901-05-05 12:34:56.1096,1901-12-25 +1901-05-05 12:34:56.1097,1901-12-25 +1901-05-05 12:34:56.1098,1901-12-25 +1901-05-05 12:34:56.1099,1901-12-25 +1901-05-05 12:34:56.11,1901-12-25 +1901-05-05 12:34:56.1101,1901-12-25 +1901-05-05 12:34:56.1102,1901-12-25 +1901-05-05 12:34:56.1103,1901-12-25 +1901-05-05 12:34:56.1104,1901-12-25 +1901-05-05 12:34:56.1105,1901-12-25 +1901-05-05 12:34:56.1106,1901-12-25 +1901-05-05 12:34:56.1107,1901-12-25 +1901-05-05 12:34:56.1108,1901-12-25 +1901-05-05 12:34:56.1109,1901-12-25 +1901-05-05 12:34:56.111,1901-12-25 +1901-05-05 12:34:56.1111,1901-12-25 +1901-05-05 12:34:56.1112,1901-12-25 +1901-05-05 12:34:56.1113,1901-12-25 +1901-05-05 12:34:56.1114,1901-12-25 +1901-05-05 12:34:56.1115,1901-12-25 +1901-05-05 12:34:56.1116,1901-12-25 +1901-05-05 12:34:56.1117,1901-12-25 +1901-05-05 12:34:56.1118,1901-12-25 +1901-05-05 12:34:56.1119,1901-12-25 +1901-05-05 12:34:56.112,1901-12-25 +1901-05-05 12:34:56.1121,1901-12-25 +1901-05-05 12:34:56.1122,1901-12-25 +1901-05-05 12:34:56.1123,1901-12-25 +1901-05-05 12:34:56.1124,1901-12-25 +1901-05-05 12:34:56.1125,1901-12-25 +1901-05-05 12:34:56.1126,1901-12-25 +1901-05-05 12:34:56.1127,1901-12-25 +1901-05-05 12:34:56.1128,1901-12-25 +1901-05-05 12:34:56.1129,1901-12-25 +1901-05-05 12:34:56.113,1901-12-25 +1901-05-05 12:34:56.1131,1901-12-25 +1901-05-05 12:34:56.1132,1901-12-25 +1901-05-05 12:34:56.1133,1901-12-25 +1901-05-05 12:34:56.1134,1901-12-25 +1901-05-05 12:34:56.1135,1901-12-25 +1901-05-05 12:34:56.1136,1901-12-25 +1901-05-05 12:34:56.1137,1901-12-25 +1901-05-05 12:34:56.1138,1901-12-25 +1901-05-05 12:34:56.1139,1901-12-25 +1901-05-05 12:34:56.114,1901-12-25 +1901-05-05 12:34:56.1141,1901-12-25 +1901-05-05 12:34:56.1142,1901-12-25 +1901-05-05 12:34:56.1143,1901-12-25 +1901-05-05 12:34:56.1144,1901-12-25 +1901-05-05 12:34:56.1145,1901-12-25 +1901-05-05 12:34:56.1146,1901-12-25 +1901-05-05 12:34:56.1147,1901-12-25 +1901-05-05 12:34:56.1148,1901-12-25 +1901-05-05 12:34:56.1149,1901-12-25 +1901-05-05 12:34:56.115,1901-12-25 +1901-05-05 12:34:56.1151,1901-12-25 +1901-05-05 12:34:56.1152,1901-12-25 +1901-05-05 12:34:56.1153,1901-12-25 +1901-05-05 12:34:56.1154,1901-12-25 +1901-05-05 12:34:56.1155,1901-12-25 +1901-05-05 12:34:56.1156,1901-12-25 +1901-05-05 12:34:56.1157,1901-12-25 +1901-05-05 12:34:56.1158,1901-12-25 +1901-05-05 12:34:56.1159,1901-12-25 +1901-05-05 12:34:56.116,1901-12-25 +1901-05-05 12:34:56.1161,1901-12-25 +1901-05-05 12:34:56.1162,1901-12-25 +1901-05-05 12:34:56.1163,1901-12-25 +1901-05-05 12:34:56.1164,1901-12-25 +1901-05-05 12:34:56.1165,1901-12-25 +1901-05-05 12:34:56.1166,1901-12-25 +1901-05-05 12:34:56.1167,1901-12-25 +1901-05-05 12:34:56.1168,1901-12-25 +1901-05-05 12:34:56.1169,1901-12-25 +1901-05-05 12:34:56.117,1901-12-25 +1901-05-05 12:34:56.1171,1901-12-25 +1901-05-05 12:34:56.1172,1901-12-25 +1901-05-05 12:34:56.1173,1901-12-25 +1901-05-05 12:34:56.1174,1901-12-25 +1901-05-05 12:34:56.1175,1901-12-25 +1901-05-05 12:34:56.1176,1901-12-25 +1901-05-05 12:34:56.1177,1901-12-25 +1901-05-05 12:34:56.1178,1901-12-25 +1901-05-05 12:34:56.1179,1901-12-25 +1901-05-05 12:34:56.118,1901-12-25 +1901-05-05 12:34:56.1181,1901-12-25 +1901-05-05 12:34:56.1182,1901-12-25 +1901-05-05 12:34:56.1183,1901-12-25 +1901-05-05 12:34:56.1184,1901-12-25 +1901-05-05 12:34:56.1185,1901-12-25 +1901-05-05 12:34:56.1186,1901-12-25 +1901-05-05 12:34:56.1187,1901-12-25 +1901-05-05 12:34:56.1188,1901-12-25 +1901-05-05 12:34:56.1189,1901-12-25 +1901-05-05 12:34:56.119,1901-12-25 +1901-05-05 12:34:56.1191,1901-12-25 +1901-05-05 12:34:56.1192,1901-12-25 +1901-05-05 12:34:56.1193,1901-12-25 +1901-05-05 12:34:56.1194,1901-12-25 +1901-05-05 12:34:56.1195,1901-12-25 +1901-05-05 12:34:56.1196,1901-12-25 +1901-05-05 12:34:56.1197,1901-12-25 +1901-05-05 12:34:56.1198,1901-12-25 +1901-05-05 12:34:56.1199,1901-12-25 +1901-05-05 12:34:56.12,1901-12-25 +1901-05-05 12:34:56.1201,1901-12-25 +1901-05-05 12:34:56.1202,1901-12-25 +1901-05-05 12:34:56.1203,1901-12-25 +1901-05-05 12:34:56.1204,1901-12-25 +1901-05-05 12:34:56.1205,1901-12-25 +1901-05-05 12:34:56.1206,1901-12-25 +1901-05-05 12:34:56.1207,1901-12-25 +1901-05-05 12:34:56.1208,1901-12-25 +1901-05-05 12:34:56.1209,1901-12-25 +1901-05-05 12:34:56.121,1901-12-25 +1901-05-05 12:34:56.1211,1901-12-25 +1901-05-05 12:34:56.1212,1901-12-25 +1901-05-05 12:34:56.1213,1901-12-25 +1901-05-05 12:34:56.1214,1901-12-25 +1901-05-05 12:34:56.1215,1901-12-25 +1901-05-05 12:34:56.1216,1901-12-25 +1901-05-05 12:34:56.1217,1901-12-25 +1901-05-05 12:34:56.1218,1901-12-25 +1901-05-05 12:34:56.1219,1901-12-25 +1901-05-05 12:34:56.122,1901-12-25 +1901-05-05 12:34:56.1221,1901-12-25 +1901-05-05 12:34:56.1222,1901-12-25 +1901-05-05 12:34:56.1223,1901-12-25 +1901-05-05 12:34:56.1224,1901-12-25 +1901-05-05 12:34:56.1225,1901-12-25 +1901-05-05 12:34:56.1226,1901-12-25 +1901-05-05 12:34:56.1227,1901-12-25 +1901-05-05 12:34:56.1228,1901-12-25 +1901-05-05 12:34:56.1229,1901-12-25 +1901-05-05 12:34:56.123,1901-12-25 +1901-05-05 12:34:56.1231,1901-12-25 +1901-05-05 12:34:56.1232,1901-12-25 +1901-05-05 12:34:56.1233,1901-12-25 +1901-05-05 12:34:56.1234,1901-12-25 +1901-05-05 12:34:56.1235,1901-12-25 +1901-05-05 12:34:56.1236,1901-12-25 +1901-05-05 12:34:56.1237,1901-12-25 +1901-05-05 12:34:56.1238,1901-12-25 +1901-05-05 12:34:56.1239,1901-12-25 +1901-05-05 12:34:56.124,1901-12-25 +1901-05-05 12:34:56.1241,1901-12-25 +1901-05-05 12:34:56.1242,1901-12-25 +1901-05-05 12:34:56.1243,1901-12-25 +1901-05-05 12:34:56.1244,1901-12-25 +1901-05-05 12:34:56.1245,1901-12-25 +1901-05-05 12:34:56.1246,1901-12-25 +1901-05-05 12:34:56.1247,1901-12-25 +1901-05-05 12:34:56.1248,1901-12-25 +1901-05-05 12:34:56.1249,1901-12-25 +1901-05-05 12:34:56.125,1901-12-25 +1901-05-05 12:34:56.1251,1901-12-25 +1901-05-05 12:34:56.1252,1901-12-25 +1901-05-05 12:34:56.1253,1901-12-25 +1901-05-05 12:34:56.1254,1901-12-25 +1901-05-05 12:34:56.1255,1901-12-25 +1901-05-05 12:34:56.1256,1901-12-25 +1901-05-05 12:34:56.1257,1901-12-25 +1901-05-05 12:34:56.1258,1901-12-25 +1901-05-05 12:34:56.1259,1901-12-25 +1901-05-05 12:34:56.126,1901-12-25 +1901-05-05 12:34:56.1261,1901-12-25 +1901-05-05 12:34:56.1262,1901-12-25 +1901-05-05 12:34:56.1263,1901-12-25 +1901-05-05 12:34:56.1264,1901-12-25 +1901-05-05 12:34:56.1265,1901-12-25 +1901-05-05 12:34:56.1266,1901-12-25 +1901-05-05 12:34:56.1267,1901-12-25 +1901-05-05 12:34:56.1268,1901-12-25 +1901-05-05 12:34:56.1269,1901-12-25 +1901-05-05 12:34:56.127,1901-12-25 +1901-05-05 12:34:56.1271,1901-12-25 +1901-05-05 12:34:56.1272,1901-12-25 +1901-05-05 12:34:56.1273,1901-12-25 +1901-05-05 12:34:56.1274,1901-12-25 +1901-05-05 12:34:56.1275,1901-12-25 +1901-05-05 12:34:56.1276,1901-12-25 +1901-05-05 12:34:56.1277,1901-12-25 +1901-05-05 12:34:56.1278,1901-12-25 +1901-05-05 12:34:56.1279,1901-12-25 +1901-05-05 12:34:56.128,1901-12-25 +1901-05-05 12:34:56.1281,1901-12-25 +1901-05-05 12:34:56.1282,1901-12-25 +1901-05-05 12:34:56.1283,1901-12-25 +1901-05-05 12:34:56.1284,1901-12-25 +1901-05-05 12:34:56.1285,1901-12-25 +1901-05-05 12:34:56.1286,1901-12-25 +1901-05-05 12:34:56.1287,1901-12-25 +1901-05-05 12:34:56.1288,1901-12-25 +1901-05-05 12:34:56.1289,1901-12-25 +1901-05-05 12:34:56.129,1901-12-25 +1901-05-05 12:34:56.1291,1901-12-25 +1901-05-05 12:34:56.1292,1901-12-25 +1901-05-05 12:34:56.1293,1901-12-25 +1901-05-05 12:34:56.1294,1901-12-25 +1901-05-05 12:34:56.1295,1901-12-25 +1901-05-05 12:34:56.1296,1901-12-25 +1901-05-05 12:34:56.1297,1901-12-25 +1901-05-05 12:34:56.1298,1901-12-25 +1901-05-05 12:34:56.1299,1901-12-25 +1901-05-05 12:34:56.13,1901-12-25 +1901-05-05 12:34:56.1301,1901-12-25 +1901-05-05 12:34:56.1302,1901-12-25 +1901-05-05 12:34:56.1303,1901-12-25 +1901-05-05 12:34:56.1304,1901-12-25 +1901-05-05 12:34:56.1305,1901-12-25 +1901-05-05 12:34:56.1306,1901-12-25 +1901-05-05 12:34:56.1307,1901-12-25 +1901-05-05 12:34:56.1308,1901-12-25 +1901-05-05 12:34:56.1309,1901-12-25 +1901-05-05 12:34:56.131,1901-12-25 +1901-05-05 12:34:56.1311,1901-12-25 +1901-05-05 12:34:56.1312,1901-12-25 +1901-05-05 12:34:56.1313,1901-12-25 +1901-05-05 12:34:56.1314,1901-12-25 +1901-05-05 12:34:56.1315,1901-12-25 +1901-05-05 12:34:56.1316,1901-12-25 +1901-05-05 12:34:56.1317,1901-12-25 +1901-05-05 12:34:56.1318,1901-12-25 +1901-05-05 12:34:56.1319,1901-12-25 +1901-05-05 12:34:56.132,1901-12-25 +1901-05-05 12:34:56.1321,1901-12-25 +1901-05-05 12:34:56.1322,1901-12-25 +1901-05-05 12:34:56.1323,1901-12-25 +1901-05-05 12:34:56.1324,1901-12-25 +1901-05-05 12:34:56.1325,1901-12-25 +1901-05-05 12:34:56.1326,1901-12-25 +1901-05-05 12:34:56.1327,1901-12-25 +1901-05-05 12:34:56.1328,1901-12-25 +1901-05-05 12:34:56.1329,1901-12-25 +1901-05-05 12:34:56.133,1901-12-25 +1901-05-05 12:34:56.1331,1901-12-25 +1901-05-05 12:34:56.1332,1901-12-25 +1901-05-05 12:34:56.1333,1901-12-25 +1901-05-05 12:34:56.1334,1901-12-25 +1901-05-05 12:34:56.1335,1901-12-25 +1901-05-05 12:34:56.1336,1901-12-25 +1901-05-05 12:34:56.1337,1901-12-25 +1901-05-05 12:34:56.1338,1901-12-25 +1901-05-05 12:34:56.1339,1901-12-25 +1901-05-05 12:34:56.134,1901-12-25 +1901-05-05 12:34:56.1341,1901-12-25 +1901-05-05 12:34:56.1342,1901-12-25 +1901-05-05 12:34:56.1343,1901-12-25 +1901-05-05 12:34:56.1344,1901-12-25 +1901-05-05 12:34:56.1345,1901-12-25 +1901-05-05 12:34:56.1346,1901-12-25 +1901-05-05 12:34:56.1347,1901-12-25 +1901-05-05 12:34:56.1348,1901-12-25 +1901-05-05 12:34:56.1349,1901-12-25 +1901-05-05 12:34:56.135,1901-12-25 +1901-05-05 12:34:56.1351,1901-12-25 +1901-05-05 12:34:56.1352,1901-12-25 +1901-05-05 12:34:56.1353,1901-12-25 +1901-05-05 12:34:56.1354,1901-12-25 +1901-05-05 12:34:56.1355,1901-12-25 +1901-05-05 12:34:56.1356,1901-12-25 +1901-05-05 12:34:56.1357,1901-12-25 +1901-05-05 12:34:56.1358,1901-12-25 +1901-05-05 12:34:56.1359,1901-12-25 +1901-05-05 12:34:56.136,1901-12-25 +1901-05-05 12:34:56.1361,1901-12-25 +1901-05-05 12:34:56.1362,1901-12-25 +1901-05-05 12:34:56.1363,1901-12-25 +1901-05-05 12:34:56.1364,1901-12-25 +1901-05-05 12:34:56.1365,1901-12-25 +1901-05-05 12:34:56.1366,1901-12-25 +1901-05-05 12:34:56.1367,1901-12-25 +1901-05-05 12:34:56.1368,1901-12-25 +1901-05-05 12:34:56.1369,1901-12-25 +1901-05-05 12:34:56.137,1901-12-25 +1901-05-05 12:34:56.1371,1901-12-25 +1901-05-05 12:34:56.1372,1901-12-25 +1901-05-05 12:34:56.1373,1901-12-25 +1901-05-05 12:34:56.1374,1901-12-25 +1901-05-05 12:34:56.1375,1901-12-25 +1901-05-05 12:34:56.1376,1901-12-25 +1901-05-05 12:34:56.1377,1901-12-25 +1901-05-05 12:34:56.1378,1901-12-25 +1901-05-05 12:34:56.1379,1901-12-25 +1901-05-05 12:34:56.138,1901-12-25 +1901-05-05 12:34:56.1381,1901-12-25 +1901-05-05 12:34:56.1382,1901-12-25 +1901-05-05 12:34:56.1383,1901-12-25 +1901-05-05 12:34:56.1384,1901-12-25 +1901-05-05 12:34:56.1385,1901-12-25 +1901-05-05 12:34:56.1386,1901-12-25 +1901-05-05 12:34:56.1387,1901-12-25 +1901-05-05 12:34:56.1388,1901-12-25 +1901-05-05 12:34:56.1389,1901-12-25 +1901-05-05 12:34:56.139,1901-12-25 +1901-05-05 12:34:56.1391,1901-12-25 +1901-05-05 12:34:56.1392,1901-12-25 +1901-05-05 12:34:56.1393,1901-12-25 +1901-05-05 12:34:56.1394,1901-12-25 +1901-05-05 12:34:56.1395,1901-12-25 +1901-05-05 12:34:56.1396,1901-12-25 +1901-05-05 12:34:56.1397,1901-12-25 +1901-05-05 12:34:56.1398,1901-12-25 +1901-05-05 12:34:56.1399,1901-12-25 +1901-05-05 12:34:56.14,1901-12-25 +1901-05-05 12:34:56.1401,1901-12-25 +1901-05-05 12:34:56.1402,1901-12-25 +1901-05-05 12:34:56.1403,1901-12-25 +1901-05-05 12:34:56.1404,1901-12-25 +1901-05-05 12:34:56.1405,1901-12-25 +1901-05-05 12:34:56.1406,1901-12-25 +1901-05-05 12:34:56.1407,1901-12-25 +1901-05-05 12:34:56.1408,1901-12-25 +1901-05-05 12:34:56.1409,1901-12-25 +1901-05-05 12:34:56.141,1901-12-25 +1901-05-05 12:34:56.1411,1901-12-25 +1901-05-05 12:34:56.1412,1901-12-25 +1901-05-05 12:34:56.1413,1901-12-25 +1901-05-05 12:34:56.1414,1901-12-25 +1901-05-05 12:34:56.1415,1901-12-25 +1901-05-05 12:34:56.1416,1901-12-25 +1901-05-05 12:34:56.1417,1901-12-25 +1901-05-05 12:34:56.1418,1901-12-25 +1901-05-05 12:34:56.1419,1901-12-25 +1901-05-05 12:34:56.142,1901-12-25 +1901-05-05 12:34:56.1421,1901-12-25 +1901-05-05 12:34:56.1422,1901-12-25 +1901-05-05 12:34:56.1423,1901-12-25 +1901-05-05 12:34:56.1424,1901-12-25 +1901-05-05 12:34:56.1425,1901-12-25 +1901-05-05 12:34:56.1426,1901-12-25 +1901-05-05 12:34:56.1427,1901-12-25 +1901-05-05 12:34:56.1428,1901-12-25 +1901-05-05 12:34:56.1429,1901-12-25 +1901-05-05 12:34:56.143,1901-12-25 +1901-05-05 12:34:56.1431,1901-12-25 +1901-05-05 12:34:56.1432,1901-12-25 +1901-05-05 12:34:56.1433,1901-12-25 +1901-05-05 12:34:56.1434,1901-12-25 +1901-05-05 12:34:56.1435,1901-12-25 +1901-05-05 12:34:56.1436,1901-12-25 +1901-05-05 12:34:56.1437,1901-12-25 +1901-05-05 12:34:56.1438,1901-12-25 +1901-05-05 12:34:56.1439,1901-12-25 +1901-05-05 12:34:56.144,1901-12-25 +1901-05-05 12:34:56.1441,1901-12-25 +1901-05-05 12:34:56.1442,1901-12-25 +1901-05-05 12:34:56.1443,1901-12-25 +1901-05-05 12:34:56.1444,1901-12-25 +1901-05-05 12:34:56.1445,1901-12-25 +1901-05-05 12:34:56.1446,1901-12-25 +1901-05-05 12:34:56.1447,1901-12-25 +1901-05-05 12:34:56.1448,1901-12-25 +1901-05-05 12:34:56.1449,1901-12-25 +1901-05-05 12:34:56.145,1901-12-25 +1901-05-05 12:34:56.1451,1901-12-25 +1901-05-05 12:34:56.1452,1901-12-25 +1901-05-05 12:34:56.1453,1901-12-25 +1901-05-05 12:34:56.1454,1901-12-25 +1901-05-05 12:34:56.1455,1901-12-25 +1901-05-05 12:34:56.1456,1901-12-25 +1901-05-05 12:34:56.1457,1901-12-25 +1901-05-05 12:34:56.1458,1901-12-25 +1901-05-05 12:34:56.1459,1901-12-25 +1901-05-05 12:34:56.146,1901-12-25 +1901-05-05 12:34:56.1461,1901-12-25 +1901-05-05 12:34:56.1462,1901-12-25 +1901-05-05 12:34:56.1463,1901-12-25 +1901-05-05 12:34:56.1464,1901-12-25 +1901-05-05 12:34:56.1465,1901-12-25 +1901-05-05 12:34:56.1466,1901-12-25 +1901-05-05 12:34:56.1467,1901-12-25 +1901-05-05 12:34:56.1468,1901-12-25 +1901-05-05 12:34:56.1469,1901-12-25 +1901-05-05 12:34:56.147,1901-12-25 +1901-05-05 12:34:56.1471,1901-12-25 +1901-05-05 12:34:56.1472,1901-12-25 +1901-05-05 12:34:56.1473,1901-12-25 +1901-05-05 12:34:56.1474,1901-12-25 +1901-05-05 12:34:56.1475,1901-12-25 +1901-05-05 12:34:56.1476,1901-12-25 +1901-05-05 12:34:56.1477,1901-12-25 +1901-05-05 12:34:56.1478,1901-12-25 +1901-05-05 12:34:56.1479,1901-12-25 +1901-05-05 12:34:56.148,1901-12-25 +1901-05-05 12:34:56.1481,1901-12-25 +1901-05-05 12:34:56.1482,1901-12-25 +1901-05-05 12:34:56.1483,1901-12-25 +1901-05-05 12:34:56.1484,1901-12-25 +1901-05-05 12:34:56.1485,1901-12-25 +1901-05-05 12:34:56.1486,1901-12-25 +1901-05-05 12:34:56.1487,1901-12-25 +1901-05-05 12:34:56.1488,1901-12-25 +1901-05-05 12:34:56.1489,1901-12-25 +1901-05-05 12:34:56.149,1901-12-25 +1901-05-05 12:34:56.1491,1901-12-25 +1901-05-05 12:34:56.1492,1901-12-25 +1901-05-05 12:34:56.1493,1901-12-25 +1901-05-05 12:34:56.1494,1901-12-25 +1901-05-05 12:34:56.1495,1901-12-25 +1901-05-05 12:34:56.1496,1901-12-25 +1901-05-05 12:34:56.1497,1901-12-25 +1901-05-05 12:34:56.1498,1901-12-25 +1901-05-05 12:34:56.1499,1901-12-25 +1901-05-05 12:34:56.15,1901-12-25 +1901-05-05 12:34:56.1501,1901-12-25 +1901-05-05 12:34:56.1502,1901-12-25 +1901-05-05 12:34:56.1503,1901-12-25 +1901-05-05 12:34:56.1504,1901-12-25 +1901-05-05 12:34:56.1505,1901-12-25 +1901-05-05 12:34:56.1506,1901-12-25 +1901-05-05 12:34:56.1507,1901-12-25 +1901-05-05 12:34:56.1508,1901-12-25 +1901-05-05 12:34:56.1509,1901-12-25 +1901-05-05 12:34:56.151,1901-12-25 +1901-05-05 12:34:56.1511,1901-12-25 +1901-05-05 12:34:56.1512,1901-12-25 +1901-05-05 12:34:56.1513,1901-12-25 +1901-05-05 12:34:56.1514,1901-12-25 +1901-05-05 12:34:56.1515,1901-12-25 +1901-05-05 12:34:56.1516,1901-12-25 +1901-05-05 12:34:56.1517,1901-12-25 +1901-05-05 12:34:56.1518,1901-12-25 +1901-05-05 12:34:56.1519,1901-12-25 +1901-05-05 12:34:56.152,1901-12-25 +1901-05-05 12:34:56.1521,1901-12-25 +1901-05-05 12:34:56.1522,1901-12-25 +1901-05-05 12:34:56.1523,1901-12-25 +1901-05-05 12:34:56.1524,1901-12-25 +1901-05-05 12:34:56.1525,1901-12-25 +1901-05-05 12:34:56.1526,1901-12-25 +1901-05-05 12:34:56.1527,1901-12-25 +1901-05-05 12:34:56.1528,1901-12-25 +1901-05-05 12:34:56.1529,1901-12-25 +1901-05-05 12:34:56.153,1901-12-25 +1901-05-05 12:34:56.1531,1901-12-25 +1901-05-05 12:34:56.1532,1901-12-25 +1901-05-05 12:34:56.1533,1901-12-25 +1901-05-05 12:34:56.1534,1901-12-25 +1901-05-05 12:34:56.1535,1901-12-25 +1901-05-05 12:34:56.1536,1901-12-25 +1901-05-05 12:34:56.1537,1901-12-25 +1901-05-05 12:34:56.1538,1901-12-25 +1901-05-05 12:34:56.1539,1901-12-25 +1901-05-05 12:34:56.154,1901-12-25 +1901-05-05 12:34:56.1541,1901-12-25 +1901-05-05 12:34:56.1542,1901-12-25 +1901-05-05 12:34:56.1543,1901-12-25 +1901-05-05 12:34:56.1544,1901-12-25 +1901-05-05 12:34:56.1545,1901-12-25 +1901-05-05 12:34:56.1546,1901-12-25 +1901-05-05 12:34:56.1547,1901-12-25 +1901-05-05 12:34:56.1548,1901-12-25 +1901-05-05 12:34:56.1549,1901-12-25 +1901-05-05 12:34:56.155,1901-12-25 +1901-05-05 12:34:56.1551,1901-12-25 +1901-05-05 12:34:56.1552,1901-12-25 +1901-05-05 12:34:56.1553,1901-12-25 +1901-05-05 12:34:56.1554,1901-12-25 +1901-05-05 12:34:56.1555,1901-12-25 +1901-05-05 12:34:56.1556,1901-12-25 +1901-05-05 12:34:56.1557,1901-12-25 +1901-05-05 12:34:56.1558,1901-12-25 +1901-05-05 12:34:56.1559,1901-12-25 +1901-05-05 12:34:56.156,1901-12-25 +1901-05-05 12:34:56.1561,1901-12-25 +1901-05-05 12:34:56.1562,1901-12-25 +1901-05-05 12:34:56.1563,1901-12-25 +1901-05-05 12:34:56.1564,1901-12-25 +1901-05-05 12:34:56.1565,1901-12-25 +1901-05-05 12:34:56.1566,1901-12-25 +1901-05-05 12:34:56.1567,1901-12-25 +1901-05-05 12:34:56.1568,1901-12-25 +1901-05-05 12:34:56.1569,1901-12-25 +1901-05-05 12:34:56.157,1901-12-25 +1901-05-05 12:34:56.1571,1901-12-25 +1901-05-05 12:34:56.1572,1901-12-25 +1901-05-05 12:34:56.1573,1901-12-25 +1901-05-05 12:34:56.1574,1901-12-25 +1901-05-05 12:34:56.1575,1901-12-25 +1901-05-05 12:34:56.1576,1901-12-25 +1901-05-05 12:34:56.1577,1901-12-25 +1901-05-05 12:34:56.1578,1901-12-25 +1901-05-05 12:34:56.1579,1901-12-25 +1901-05-05 12:34:56.158,1901-12-25 +1901-05-05 12:34:56.1581,1901-12-25 +1901-05-05 12:34:56.1582,1901-12-25 +1901-05-05 12:34:56.1583,1901-12-25 +1901-05-05 12:34:56.1584,1901-12-25 +1901-05-05 12:34:56.1585,1901-12-25 +1901-05-05 12:34:56.1586,1901-12-25 +1901-05-05 12:34:56.1587,1901-12-25 +1901-05-05 12:34:56.1588,1901-12-25 +1901-05-05 12:34:56.1589,1901-12-25 +1901-05-05 12:34:56.159,1901-12-25 +1901-05-05 12:34:56.1591,1901-12-25 +1901-05-05 12:34:56.1592,1901-12-25 +1901-05-05 12:34:56.1593,1901-12-25 +1901-05-05 12:34:56.1594,1901-12-25 +1901-05-05 12:34:56.1595,1901-12-25 +1901-05-05 12:34:56.1596,1901-12-25 +1901-05-05 12:34:56.1597,1901-12-25 +1901-05-05 12:34:56.1598,1901-12-25 +1901-05-05 12:34:56.1599,1901-12-25 +1901-05-05 12:34:56.16,1901-12-25 +1901-05-05 12:34:56.1601,1901-12-25 +1901-05-05 12:34:56.1602,1901-12-25 +1901-05-05 12:34:56.1603,1901-12-25 +1901-05-05 12:34:56.1604,1901-12-25 +1901-05-05 12:34:56.1605,1901-12-25 +1901-05-05 12:34:56.1606,1901-12-25 +1901-05-05 12:34:56.1607,1901-12-25 +1901-05-05 12:34:56.1608,1901-12-25 +1901-05-05 12:34:56.1609,1901-12-25 +1901-05-05 12:34:56.161,1901-12-25 +1901-05-05 12:34:56.1611,1901-12-25 +1901-05-05 12:34:56.1612,1901-12-25 +1901-05-05 12:34:56.1613,1901-12-25 +1901-05-05 12:34:56.1614,1901-12-25 +1901-05-05 12:34:56.1615,1901-12-25 +1901-05-05 12:34:56.1616,1901-12-25 +1901-05-05 12:34:56.1617,1901-12-25 +1901-05-05 12:34:56.1618,1901-12-25 +1901-05-05 12:34:56.1619,1901-12-25 +1901-05-05 12:34:56.162,1901-12-25 +1901-05-05 12:34:56.1621,1901-12-25 +1901-05-05 12:34:56.1622,1901-12-25 +1901-05-05 12:34:56.1623,1901-12-25 +1901-05-05 12:34:56.1624,1901-12-25 +1901-05-05 12:34:56.1625,1901-12-25 +1901-05-05 12:34:56.1626,1901-12-25 +1901-05-05 12:34:56.1627,1901-12-25 +1901-05-05 12:34:56.1628,1901-12-25 +1901-05-05 12:34:56.1629,1901-12-25 +1901-05-05 12:34:56.163,1901-12-25 +1901-05-05 12:34:56.1631,1901-12-25 +1901-05-05 12:34:56.1632,1901-12-25 +1901-05-05 12:34:56.1633,1901-12-25 +1901-05-05 12:34:56.1634,1901-12-25 +1901-05-05 12:34:56.1635,1901-12-25 +1901-05-05 12:34:56.1636,1901-12-25 +1901-05-05 12:34:56.1637,1901-12-25 +1901-05-05 12:34:56.1638,1901-12-25 +1901-05-05 12:34:56.1639,1901-12-25 +1901-05-05 12:34:56.164,1901-12-25 +1901-05-05 12:34:56.1641,1901-12-25 +1901-05-05 12:34:56.1642,1901-12-25 +1901-05-05 12:34:56.1643,1901-12-25 +1901-05-05 12:34:56.1644,1901-12-25 +1901-05-05 12:34:56.1645,1901-12-25 +1901-05-05 12:34:56.1646,1901-12-25 +1901-05-05 12:34:56.1647,1901-12-25 +1901-05-05 12:34:56.1648,1901-12-25 +1901-05-05 12:34:56.1649,1901-12-25 +1901-05-05 12:34:56.165,1901-12-25 +1901-05-05 12:34:56.1651,1901-12-25 +1901-05-05 12:34:56.1652,1901-12-25 +1901-05-05 12:34:56.1653,1901-12-25 +1901-05-05 12:34:56.1654,1901-12-25 +1901-05-05 12:34:56.1655,1901-12-25 +1901-05-05 12:34:56.1656,1901-12-25 +1901-05-05 12:34:56.1657,1901-12-25 +1901-05-05 12:34:56.1658,1901-12-25 +1901-05-05 12:34:56.1659,1901-12-25 +1901-05-05 12:34:56.166,1901-12-25 +1901-05-05 12:34:56.1661,1901-12-25 +1901-05-05 12:34:56.1662,1901-12-25 +1901-05-05 12:34:56.1663,1901-12-25 +1901-05-05 12:34:56.1664,1901-12-25 +1901-05-05 12:34:56.1665,1901-12-25 +1901-05-05 12:34:56.1666,1901-12-25 +1901-05-05 12:34:56.1667,1901-12-25 +1901-05-05 12:34:56.1668,1901-12-25 +1901-05-05 12:34:56.1669,1901-12-25 +1901-05-05 12:34:56.167,1901-12-25 +1901-05-05 12:34:56.1671,1901-12-25 +1901-05-05 12:34:56.1672,1901-12-25 +1901-05-05 12:34:56.1673,1901-12-25 +1901-05-05 12:34:56.1674,1901-12-25 +1901-05-05 12:34:56.1675,1901-12-25 +1901-05-05 12:34:56.1676,1901-12-25 +1901-05-05 12:34:56.1677,1901-12-25 +1901-05-05 12:34:56.1678,1901-12-25 +1901-05-05 12:34:56.1679,1901-12-25 +1901-05-05 12:34:56.168,1901-12-25 +1901-05-05 12:34:56.1681,1901-12-25 +1901-05-05 12:34:56.1682,1901-12-25 +1901-05-05 12:34:56.1683,1901-12-25 +1901-05-05 12:34:56.1684,1901-12-25 +1901-05-05 12:34:56.1685,1901-12-25 +1901-05-05 12:34:56.1686,1901-12-25 +1901-05-05 12:34:56.1687,1901-12-25 +1901-05-05 12:34:56.1688,1901-12-25 +1901-05-05 12:34:56.1689,1901-12-25 +1901-05-05 12:34:56.169,1901-12-25 +1901-05-05 12:34:56.1691,1901-12-25 +1901-05-05 12:34:56.1692,1901-12-25 +1901-05-05 12:34:56.1693,1901-12-25 +1901-05-05 12:34:56.1694,1901-12-25 +1901-05-05 12:34:56.1695,1901-12-25 +1901-05-05 12:34:56.1696,1901-12-25 +1901-05-05 12:34:56.1697,1901-12-25 +1901-05-05 12:34:56.1698,1901-12-25 +1901-05-05 12:34:56.1699,1901-12-25 +1901-05-05 12:34:56.17,1901-12-25 +1901-05-05 12:34:56.1701,1901-12-25 +1901-05-05 12:34:56.1702,1901-12-25 +1901-05-05 12:34:56.1703,1901-12-25 +1901-05-05 12:34:56.1704,1901-12-25 +1901-05-05 12:34:56.1705,1901-12-25 +1901-05-05 12:34:56.1706,1901-12-25 +1901-05-05 12:34:56.1707,1901-12-25 +1901-05-05 12:34:56.1708,1901-12-25 +1901-05-05 12:34:56.1709,1901-12-25 +1901-05-05 12:34:56.171,1901-12-25 +1901-05-05 12:34:56.1711,1901-12-25 +1901-05-05 12:34:56.1712,1901-12-25 +1901-05-05 12:34:56.1713,1901-12-25 +1901-05-05 12:34:56.1714,1901-12-25 +1901-05-05 12:34:56.1715,1901-12-25 +1901-05-05 12:34:56.1716,1901-12-25 +1901-05-05 12:34:56.1717,1901-12-25 +1901-05-05 12:34:56.1718,1901-12-25 +1901-05-05 12:34:56.1719,1901-12-25 +1901-05-05 12:34:56.172,1901-12-25 +1901-05-05 12:34:56.1721,1901-12-25 +1901-05-05 12:34:56.1722,1901-12-25 +1901-05-05 12:34:56.1723,1901-12-25 +1901-05-05 12:34:56.1724,1901-12-25 +1901-05-05 12:34:56.1725,1901-12-25 +1901-05-05 12:34:56.1726,1901-12-25 +1901-05-05 12:34:56.1727,1901-12-25 +1901-05-05 12:34:56.1728,1901-12-25 +1901-05-05 12:34:56.1729,1901-12-25 +1901-05-05 12:34:56.173,1901-12-25 +1901-05-05 12:34:56.1731,1901-12-25 +1901-05-05 12:34:56.1732,1901-12-25 +1901-05-05 12:34:56.1733,1901-12-25 +1901-05-05 12:34:56.1734,1901-12-25 +1901-05-05 12:34:56.1735,1901-12-25 +1901-05-05 12:34:56.1736,1901-12-25 +1901-05-05 12:34:56.1737,1901-12-25 +1901-05-05 12:34:56.1738,1901-12-25 +1901-05-05 12:34:56.1739,1901-12-25 +1901-05-05 12:34:56.174,1901-12-25 +1901-05-05 12:34:56.1741,1901-12-25 +1901-05-05 12:34:56.1742,1901-12-25 +1901-05-05 12:34:56.1743,1901-12-25 +1901-05-05 12:34:56.1744,1901-12-25 +1901-05-05 12:34:56.1745,1901-12-25 +1901-05-05 12:34:56.1746,1901-12-25 +1901-05-05 12:34:56.1747,1901-12-25 +1901-05-05 12:34:56.1748,1901-12-25 +1901-05-05 12:34:56.1749,1901-12-25 +1901-05-05 12:34:56.175,1901-12-25 +1901-05-05 12:34:56.1751,1901-12-25 +1901-05-05 12:34:56.1752,1901-12-25 +1901-05-05 12:34:56.1753,1901-12-25 +1901-05-05 12:34:56.1754,1901-12-25 +1901-05-05 12:34:56.1755,1901-12-25 +1901-05-05 12:34:56.1756,1901-12-25 +1901-05-05 12:34:56.1757,1901-12-25 +1901-05-05 12:34:56.1758,1901-12-25 +1901-05-05 12:34:56.1759,1901-12-25 +1901-05-05 12:34:56.176,1901-12-25 +1901-05-05 12:34:56.1761,1901-12-25 +1901-05-05 12:34:56.1762,1901-12-25 +1901-05-05 12:34:56.1763,1901-12-25 +1901-05-05 12:34:56.1764,1901-12-25 +1901-05-05 12:34:56.1765,1901-12-25 +1901-05-05 12:34:56.1766,1901-12-25 +1901-05-05 12:34:56.1767,1901-12-25 +1901-05-05 12:34:56.1768,1901-12-25 +1901-05-05 12:34:56.1769,1901-12-25 +1901-05-05 12:34:56.177,1901-12-25 +1901-05-05 12:34:56.1771,1901-12-25 +1901-05-05 12:34:56.1772,1901-12-25 +1901-05-05 12:34:56.1773,1901-12-25 +1901-05-05 12:34:56.1774,1901-12-25 +1901-05-05 12:34:56.1775,1901-12-25 +1901-05-05 12:34:56.1776,1901-12-25 +1901-05-05 12:34:56.1777,1901-12-25 +1901-05-05 12:34:56.1778,1901-12-25 +1901-05-05 12:34:56.1779,1901-12-25 +1901-05-05 12:34:56.178,1901-12-25 +1901-05-05 12:34:56.1781,1901-12-25 +1901-05-05 12:34:56.1782,1901-12-25 +1901-05-05 12:34:56.1783,1901-12-25 +1901-05-05 12:34:56.1784,1901-12-25 +1901-05-05 12:34:56.1785,1901-12-25 +1901-05-05 12:34:56.1786,1901-12-25 +1901-05-05 12:34:56.1787,1901-12-25 +1901-05-05 12:34:56.1788,1901-12-25 +1901-05-05 12:34:56.1789,1901-12-25 +1901-05-05 12:34:56.179,1901-12-25 +1901-05-05 12:34:56.1791,1901-12-25 +1901-05-05 12:34:56.1792,1901-12-25 +1901-05-05 12:34:56.1793,1901-12-25 +1901-05-05 12:34:56.1794,1901-12-25 +1901-05-05 12:34:56.1795,1901-12-25 +1901-05-05 12:34:56.1796,1901-12-25 +1901-05-05 12:34:56.1797,1901-12-25 +1901-05-05 12:34:56.1798,1901-12-25 +1901-05-05 12:34:56.1799,1901-12-25 +1901-05-05 12:34:56.18,1901-12-25 +1901-05-05 12:34:56.1801,1901-12-25 +1901-05-05 12:34:56.1802,1901-12-25 +1901-05-05 12:34:56.1803,1901-12-25 +1901-05-05 12:34:56.1804,1901-12-25 +1901-05-05 12:34:56.1805,1901-12-25 +1901-05-05 12:34:56.1806,1901-12-25 +1901-05-05 12:34:56.1807,1901-12-25 +1901-05-05 12:34:56.1808,1901-12-25 +1901-05-05 12:34:56.1809,1901-12-25 +1901-05-05 12:34:56.181,1901-12-25 +1901-05-05 12:34:56.1811,1901-12-25 +1901-05-05 12:34:56.1812,1901-12-25 +1901-05-05 12:34:56.1813,1901-12-25 +1901-05-05 12:34:56.1814,1901-12-25 +1901-05-05 12:34:56.1815,1901-12-25 +1901-05-05 12:34:56.1816,1901-12-25 +1901-05-05 12:34:56.1817,1901-12-25 +1901-05-05 12:34:56.1818,1901-12-25 +1901-05-05 12:34:56.1819,1901-12-25 +1901-05-05 12:34:56.182,1901-12-25 +1901-05-05 12:34:56.1821,1901-12-25 +1901-05-05 12:34:56.1822,1901-12-25 +1901-05-05 12:34:56.1823,1901-12-25 +1901-05-05 12:34:56.1824,1901-12-25 +1901-05-05 12:34:56.1825,1901-12-25 +1901-05-05 12:34:56.1826,1901-12-25 +1901-05-05 12:34:56.1827,1901-12-25 +1901-05-05 12:34:56.1828,1901-12-25 +1901-05-05 12:34:56.1829,1901-12-25 +1901-05-05 12:34:56.183,1901-12-25 +1901-05-05 12:34:56.1831,1901-12-25 +1901-05-05 12:34:56.1832,1901-12-25 +1901-05-05 12:34:56.1833,1901-12-25 +1901-05-05 12:34:56.1834,1901-12-25 +1901-05-05 12:34:56.1835,1901-12-25 +1901-05-05 12:34:56.1836,1901-12-25 +1901-05-05 12:34:56.1837,1901-12-25 +1901-05-05 12:34:56.1838,1901-12-25 +1901-05-05 12:34:56.1839,1901-12-25 +1901-05-05 12:34:56.184,1901-12-25 +1901-05-05 12:34:56.1841,1901-12-25 +1901-05-05 12:34:56.1842,1901-12-25 +1901-05-05 12:34:56.1843,1901-12-25 +1901-05-05 12:34:56.1844,1901-12-25 +1901-05-05 12:34:56.1845,1901-12-25 +1901-05-05 12:34:56.1846,1901-12-25 +1901-05-05 12:34:56.1847,1901-12-25 +1901-05-05 12:34:56.1848,1901-12-25 +1901-05-05 12:34:56.1849,1901-12-25 +1901-05-05 12:34:56.185,1901-12-25 +1901-05-05 12:34:56.1851,1901-12-25 +1901-05-05 12:34:56.1852,1901-12-25 +1901-05-05 12:34:56.1853,1901-12-25 +1901-05-05 12:34:56.1854,1901-12-25 +1901-05-05 12:34:56.1855,1901-12-25 +1901-05-05 12:34:56.1856,1901-12-25 +1901-05-05 12:34:56.1857,1901-12-25 +1901-05-05 12:34:56.1858,1901-12-25 +1901-05-05 12:34:56.1859,1901-12-25 +1901-05-05 12:34:56.186,1901-12-25 +1901-05-05 12:34:56.1861,1901-12-25 +1901-05-05 12:34:56.1862,1901-12-25 +1901-05-05 12:34:56.1863,1901-12-25 +1901-05-05 12:34:56.1864,1901-12-25 +1901-05-05 12:34:56.1865,1901-12-25 +1901-05-05 12:34:56.1866,1901-12-25 +1901-05-05 12:34:56.1867,1901-12-25 +1901-05-05 12:34:56.1868,1901-12-25 +1901-05-05 12:34:56.1869,1901-12-25 +1901-05-05 12:34:56.187,1901-12-25 +1901-05-05 12:34:56.1871,1901-12-25 +1901-05-05 12:34:56.1872,1901-12-25 +1901-05-05 12:34:56.1873,1901-12-25 +1901-05-05 12:34:56.1874,1901-12-25 +1901-05-05 12:34:56.1875,1901-12-25 +1901-05-05 12:34:56.1876,1901-12-25 +1901-05-05 12:34:56.1877,1901-12-25 +1901-05-05 12:34:56.1878,1901-12-25 +1901-05-05 12:34:56.1879,1901-12-25 +1901-05-05 12:34:56.188,1901-12-25 +1901-05-05 12:34:56.1881,1901-12-25 +1901-05-05 12:34:56.1882,1901-12-25 +1901-05-05 12:34:56.1883,1901-12-25 +1901-05-05 12:34:56.1884,1901-12-25 +1901-05-05 12:34:56.1885,1901-12-25 +1901-05-05 12:34:56.1886,1901-12-25 +1901-05-05 12:34:56.1887,1901-12-25 +1901-05-05 12:34:56.1888,1901-12-25 +1901-05-05 12:34:56.1889,1901-12-25 +1901-05-05 12:34:56.189,1901-12-25 +1901-05-05 12:34:56.1891,1901-12-25 +1901-05-05 12:34:56.1892,1901-12-25 +1901-05-05 12:34:56.1893,1901-12-25 +1901-05-05 12:34:56.1894,1901-12-25 +1901-05-05 12:34:56.1895,1901-12-25 +1901-05-05 12:34:56.1896,1901-12-25 +1901-05-05 12:34:56.1897,1901-12-25 +1901-05-05 12:34:56.1898,1901-12-25 +1901-05-05 12:34:56.1899,1901-12-25 +1901-05-05 12:34:56.19,1901-12-25 +1901-05-05 12:34:56.1901,1901-12-25 +1901-05-05 12:34:56.1902,1901-12-25 +1901-05-05 12:34:56.1903,1901-12-25 +1901-05-05 12:34:56.1904,1901-12-25 +1901-05-05 12:34:56.1905,1901-12-25 +1901-05-05 12:34:56.1906,1901-12-25 +1901-05-05 12:34:56.1907,1901-12-25 +1901-05-05 12:34:56.1908,1901-12-25 +1901-05-05 12:34:56.1909,1901-12-25 +1901-05-05 12:34:56.191,1901-12-25 +1901-05-05 12:34:56.1911,1901-12-25 +1901-05-05 12:34:56.1912,1901-12-25 +1901-05-05 12:34:56.1913,1901-12-25 +1901-05-05 12:34:56.1914,1901-12-25 +1901-05-05 12:34:56.1915,1901-12-25 +1901-05-05 12:34:56.1916,1901-12-25 +1901-05-05 12:34:56.1917,1901-12-25 +1901-05-05 12:34:56.1918,1901-12-25 +1901-05-05 12:34:56.1919,1901-12-25 +1901-05-05 12:34:56.192,1901-12-25 +1901-05-05 12:34:56.1921,1901-12-25 +1901-05-05 12:34:56.1922,1901-12-25 +1901-05-05 12:34:56.1923,1901-12-25 +1901-05-05 12:34:56.1924,1901-12-25 +1901-05-05 12:34:56.1925,1901-12-25 +1901-05-05 12:34:56.1926,1901-12-25 +1901-05-05 12:34:56.1927,1901-12-25 +1901-05-05 12:34:56.1928,1901-12-25 +1901-05-05 12:34:56.1929,1901-12-25 +1901-05-05 12:34:56.193,1901-12-25 +1901-05-05 12:34:56.1931,1901-12-25 +1901-05-05 12:34:56.1932,1901-12-25 +1901-05-05 12:34:56.1933,1901-12-25 +1901-05-05 12:34:56.1934,1901-12-25 +1901-05-05 12:34:56.1935,1901-12-25 +1901-05-05 12:34:56.1936,1901-12-25 +1901-05-05 12:34:56.1937,1901-12-25 +1901-05-05 12:34:56.1938,1901-12-25 +1901-05-05 12:34:56.1939,1901-12-25 +1901-05-05 12:34:56.194,1901-12-25 +1901-05-05 12:34:56.1941,1901-12-25 +1901-05-05 12:34:56.1942,1901-12-25 +1901-05-05 12:34:56.1943,1901-12-25 +1901-05-05 12:34:56.1944,1901-12-25 +1901-05-05 12:34:56.1945,1901-12-25 +1901-05-05 12:34:56.1946,1901-12-25 +1901-05-05 12:34:56.1947,1901-12-25 +1901-05-05 12:34:56.1948,1901-12-25 +1901-05-05 12:34:56.1949,1901-12-25 +1901-05-05 12:34:56.195,1901-12-25 +1901-05-05 12:34:56.1951,1901-12-25 +1901-05-05 12:34:56.1952,1901-12-25 +1901-05-05 12:34:56.1953,1901-12-25 +1901-05-05 12:34:56.1954,1901-12-25 +1901-05-05 12:34:56.1955,1901-12-25 +1901-05-05 12:34:56.1956,1901-12-25 +1901-05-05 12:34:56.1957,1901-12-25 +1901-05-05 12:34:56.1958,1901-12-25 +1901-05-05 12:34:56.1959,1901-12-25 +1901-05-05 12:34:56.196,1901-12-25 +1901-05-05 12:34:56.1961,1901-12-25 +1901-05-05 12:34:56.1962,1901-12-25 +1901-05-05 12:34:56.1963,1901-12-25 +1901-05-05 12:34:56.1964,1901-12-25 +1901-05-05 12:34:56.1965,1901-12-25 +1901-05-05 12:34:56.1966,1901-12-25 +1901-05-05 12:34:56.1967,1901-12-25 +1901-05-05 12:34:56.1968,1901-12-25 +1901-05-05 12:34:56.1969,1901-12-25 +1901-05-05 12:34:56.197,1901-12-25 +1901-05-05 12:34:56.1971,1901-12-25 +1901-05-05 12:34:56.1972,1901-12-25 +1901-05-05 12:34:56.1973,1901-12-25 +1901-05-05 12:34:56.1974,1901-12-25 +1901-05-05 12:34:56.1975,1901-12-25 +1901-05-05 12:34:56.1976,1901-12-25 +1901-05-05 12:34:56.1977,1901-12-25 +1901-05-05 12:34:56.1978,1901-12-25 +1901-05-05 12:34:56.1979,1901-12-25 +1901-05-05 12:34:56.198,1901-12-25 +1901-05-05 12:34:56.1981,1901-12-25 +1901-05-05 12:34:56.1982,1901-12-25 +1901-05-05 12:34:56.1983,1901-12-25 +1901-05-05 12:34:56.1984,1901-12-25 +1901-05-05 12:34:56.1985,1901-12-25 +1901-05-05 12:34:56.1986,1901-12-25 +1901-05-05 12:34:56.1987,1901-12-25 +1901-05-05 12:34:56.1988,1901-12-25 +1901-05-05 12:34:56.1989,1901-12-25 +1901-05-05 12:34:56.199,1901-12-25 +1901-05-05 12:34:56.1991,1901-12-25 +1901-05-05 12:34:56.1992,1901-12-25 +1901-05-05 12:34:56.1993,1901-12-25 +1901-05-05 12:34:56.1994,1901-12-25 +1901-05-05 12:34:56.1995,1901-12-25 +1901-05-05 12:34:56.1996,1901-12-25 +1901-05-05 12:34:56.1997,1901-12-25 +1901-05-05 12:34:56.1998,1901-12-25 +1901-05-05 12:34:56.1999,1901-12-25 +1902-05-05 12:34:56.1,1902-12-25 +1902-05-05 12:34:56.1001,1902-12-25 +1902-05-05 12:34:56.1002,1902-12-25 +1902-05-05 12:34:56.1003,1902-12-25 +1902-05-05 12:34:56.1004,1902-12-25 +1902-05-05 12:34:56.1005,1902-12-25 +1902-05-05 12:34:56.1006,1902-12-25 +1902-05-05 12:34:56.1007,1902-12-25 +1902-05-05 12:34:56.1008,1902-12-25 +1902-05-05 12:34:56.1009,1902-12-25 +1902-05-05 12:34:56.101,1902-12-25 +1902-05-05 12:34:56.1011,1902-12-25 +1902-05-05 12:34:56.1012,1902-12-25 +1902-05-05 12:34:56.1013,1902-12-25 +1902-05-05 12:34:56.1014,1902-12-25 +1902-05-05 12:34:56.1015,1902-12-25 +1902-05-05 12:34:56.1016,1902-12-25 +1902-05-05 12:34:56.1017,1902-12-25 +1902-05-05 12:34:56.1018,1902-12-25 +1902-05-05 12:34:56.1019,1902-12-25 +1902-05-05 12:34:56.102,1902-12-25 +1902-05-05 12:34:56.1021,1902-12-25 +1902-05-05 12:34:56.1022,1902-12-25 +1902-05-05 12:34:56.1023,1902-12-25 +1902-05-05 12:34:56.1024,1902-12-25 +1902-05-05 12:34:56.1025,1902-12-25 +1902-05-05 12:34:56.1026,1902-12-25 +1902-05-05 12:34:56.1027,1902-12-25 +1902-05-05 12:34:56.1028,1902-12-25 +1902-05-05 12:34:56.1029,1902-12-25 +1902-05-05 12:34:56.103,1902-12-25 +1902-05-05 12:34:56.1031,1902-12-25 +1902-05-05 12:34:56.1032,1902-12-25 +1902-05-05 12:34:56.1033,1902-12-25 +1902-05-05 12:34:56.1034,1902-12-25 +1902-05-05 12:34:56.1035,1902-12-25 +1902-05-05 12:34:56.1036,1902-12-25 +1902-05-05 12:34:56.1037,1902-12-25 +1902-05-05 12:34:56.1038,1902-12-25 +1902-05-05 12:34:56.1039,1902-12-25 +1902-05-05 12:34:56.104,1902-12-25 +1902-05-05 12:34:56.1041,1902-12-25 +1902-05-05 12:34:56.1042,1902-12-25 +1902-05-05 12:34:56.1043,1902-12-25 +1902-05-05 12:34:56.1044,1902-12-25 +1902-05-05 12:34:56.1045,1902-12-25 +1902-05-05 12:34:56.1046,1902-12-25 +1902-05-05 12:34:56.1047,1902-12-25 +1902-05-05 12:34:56.1048,1902-12-25 +1902-05-05 12:34:56.1049,1902-12-25 +1902-05-05 12:34:56.105,1902-12-25 +1902-05-05 12:34:56.1051,1902-12-25 +1902-05-05 12:34:56.1052,1902-12-25 +1902-05-05 12:34:56.1053,1902-12-25 +1902-05-05 12:34:56.1054,1902-12-25 +1902-05-05 12:34:56.1055,1902-12-25 +1902-05-05 12:34:56.1056,1902-12-25 +1902-05-05 12:34:56.1057,1902-12-25 +1902-05-05 12:34:56.1058,1902-12-25 +1902-05-05 12:34:56.1059,1902-12-25 +1902-05-05 12:34:56.106,1902-12-25 +1902-05-05 12:34:56.1061,1902-12-25 +1902-05-05 12:34:56.1062,1902-12-25 +1902-05-05 12:34:56.1063,1902-12-25 +1902-05-05 12:34:56.1064,1902-12-25 +1902-05-05 12:34:56.1065,1902-12-25 +1902-05-05 12:34:56.1066,1902-12-25 +1902-05-05 12:34:56.1067,1902-12-25 +1902-05-05 12:34:56.1068,1902-12-25 +1902-05-05 12:34:56.1069,1902-12-25 +1902-05-05 12:34:56.107,1902-12-25 +1902-05-05 12:34:56.1071,1902-12-25 +1902-05-05 12:34:56.1072,1902-12-25 +1902-05-05 12:34:56.1073,1902-12-25 +1902-05-05 12:34:56.1074,1902-12-25 +1902-05-05 12:34:56.1075,1902-12-25 +1902-05-05 12:34:56.1076,1902-12-25 +1902-05-05 12:34:56.1077,1902-12-25 +1902-05-05 12:34:56.1078,1902-12-25 +1902-05-05 12:34:56.1079,1902-12-25 +1902-05-05 12:34:56.108,1902-12-25 +1902-05-05 12:34:56.1081,1902-12-25 +1902-05-05 12:34:56.1082,1902-12-25 +1902-05-05 12:34:56.1083,1902-12-25 +1902-05-05 12:34:56.1084,1902-12-25 +1902-05-05 12:34:56.1085,1902-12-25 +1902-05-05 12:34:56.1086,1902-12-25 +1902-05-05 12:34:56.1087,1902-12-25 +1902-05-05 12:34:56.1088,1902-12-25 +1902-05-05 12:34:56.1089,1902-12-25 +1902-05-05 12:34:56.109,1902-12-25 +1902-05-05 12:34:56.1091,1902-12-25 +1902-05-05 12:34:56.1092,1902-12-25 +1902-05-05 12:34:56.1093,1902-12-25 +1902-05-05 12:34:56.1094,1902-12-25 +1902-05-05 12:34:56.1095,1902-12-25 +1902-05-05 12:34:56.1096,1902-12-25 +1902-05-05 12:34:56.1097,1902-12-25 +1902-05-05 12:34:56.1098,1902-12-25 +1902-05-05 12:34:56.1099,1902-12-25 +1902-05-05 12:34:56.11,1902-12-25 +1902-05-05 12:34:56.1101,1902-12-25 +1902-05-05 12:34:56.1102,1902-12-25 +1902-05-05 12:34:56.1103,1902-12-25 +1902-05-05 12:34:56.1104,1902-12-25 +1902-05-05 12:34:56.1105,1902-12-25 +1902-05-05 12:34:56.1106,1902-12-25 +1902-05-05 12:34:56.1107,1902-12-25 +1902-05-05 12:34:56.1108,1902-12-25 +1902-05-05 12:34:56.1109,1902-12-25 +1902-05-05 12:34:56.111,1902-12-25 +1902-05-05 12:34:56.1111,1902-12-25 +1902-05-05 12:34:56.1112,1902-12-25 +1902-05-05 12:34:56.1113,1902-12-25 +1902-05-05 12:34:56.1114,1902-12-25 +1902-05-05 12:34:56.1115,1902-12-25 +1902-05-05 12:34:56.1116,1902-12-25 +1902-05-05 12:34:56.1117,1902-12-25 +1902-05-05 12:34:56.1118,1902-12-25 +1902-05-05 12:34:56.1119,1902-12-25 +1902-05-05 12:34:56.112,1902-12-25 +1902-05-05 12:34:56.1121,1902-12-25 +1902-05-05 12:34:56.1122,1902-12-25 +1902-05-05 12:34:56.1123,1902-12-25 +1902-05-05 12:34:56.1124,1902-12-25 +1902-05-05 12:34:56.1125,1902-12-25 +1902-05-05 12:34:56.1126,1902-12-25 +1902-05-05 12:34:56.1127,1902-12-25 +1902-05-05 12:34:56.1128,1902-12-25 +1902-05-05 12:34:56.1129,1902-12-25 +1902-05-05 12:34:56.113,1902-12-25 +1902-05-05 12:34:56.1131,1902-12-25 +1902-05-05 12:34:56.1132,1902-12-25 +1902-05-05 12:34:56.1133,1902-12-25 +1902-05-05 12:34:56.1134,1902-12-25 +1902-05-05 12:34:56.1135,1902-12-25 +1902-05-05 12:34:56.1136,1902-12-25 +1902-05-05 12:34:56.1137,1902-12-25 +1902-05-05 12:34:56.1138,1902-12-25 +1902-05-05 12:34:56.1139,1902-12-25 +1902-05-05 12:34:56.114,1902-12-25 +1902-05-05 12:34:56.1141,1902-12-25 +1902-05-05 12:34:56.1142,1902-12-25 +1902-05-05 12:34:56.1143,1902-12-25 +1902-05-05 12:34:56.1144,1902-12-25 +1902-05-05 12:34:56.1145,1902-12-25 +1902-05-05 12:34:56.1146,1902-12-25 +1902-05-05 12:34:56.1147,1902-12-25 +1902-05-05 12:34:56.1148,1902-12-25 +1902-05-05 12:34:56.1149,1902-12-25 +1902-05-05 12:34:56.115,1902-12-25 +1902-05-05 12:34:56.1151,1902-12-25 +1902-05-05 12:34:56.1152,1902-12-25 +1902-05-05 12:34:56.1153,1902-12-25 +1902-05-05 12:34:56.1154,1902-12-25 +1902-05-05 12:34:56.1155,1902-12-25 +1902-05-05 12:34:56.1156,1902-12-25 +1902-05-05 12:34:56.1157,1902-12-25 +1902-05-05 12:34:56.1158,1902-12-25 +1902-05-05 12:34:56.1159,1902-12-25 +1902-05-05 12:34:56.116,1902-12-25 +1902-05-05 12:34:56.1161,1902-12-25 +1902-05-05 12:34:56.1162,1902-12-25 +1902-05-05 12:34:56.1163,1902-12-25 +1902-05-05 12:34:56.1164,1902-12-25 +1902-05-05 12:34:56.1165,1902-12-25 +1902-05-05 12:34:56.1166,1902-12-25 +1902-05-05 12:34:56.1167,1902-12-25 +1902-05-05 12:34:56.1168,1902-12-25 +1902-05-05 12:34:56.1169,1902-12-25 +1902-05-05 12:34:56.117,1902-12-25 +1902-05-05 12:34:56.1171,1902-12-25 +1902-05-05 12:34:56.1172,1902-12-25 +1902-05-05 12:34:56.1173,1902-12-25 +1902-05-05 12:34:56.1174,1902-12-25 +1902-05-05 12:34:56.1175,1902-12-25 +1902-05-05 12:34:56.1176,1902-12-25 +1902-05-05 12:34:56.1177,1902-12-25 +1902-05-05 12:34:56.1178,1902-12-25 +1902-05-05 12:34:56.1179,1902-12-25 +1902-05-05 12:34:56.118,1902-12-25 +1902-05-05 12:34:56.1181,1902-12-25 +1902-05-05 12:34:56.1182,1902-12-25 +1902-05-05 12:34:56.1183,1902-12-25 +1902-05-05 12:34:56.1184,1902-12-25 +1902-05-05 12:34:56.1185,1902-12-25 +1902-05-05 12:34:56.1186,1902-12-25 +1902-05-05 12:34:56.1187,1902-12-25 +1902-05-05 12:34:56.1188,1902-12-25 +1902-05-05 12:34:56.1189,1902-12-25 +1902-05-05 12:34:56.119,1902-12-25 +1902-05-05 12:34:56.1191,1902-12-25 +1902-05-05 12:34:56.1192,1902-12-25 +1902-05-05 12:34:56.1193,1902-12-25 +1902-05-05 12:34:56.1194,1902-12-25 +1902-05-05 12:34:56.1195,1902-12-25 +1902-05-05 12:34:56.1196,1902-12-25 +1902-05-05 12:34:56.1197,1902-12-25 +1902-05-05 12:34:56.1198,1902-12-25 +1902-05-05 12:34:56.1199,1902-12-25 +1902-05-05 12:34:56.12,1902-12-25 +1902-05-05 12:34:56.1201,1902-12-25 +1902-05-05 12:34:56.1202,1902-12-25 +1902-05-05 12:34:56.1203,1902-12-25 +1902-05-05 12:34:56.1204,1902-12-25 +1902-05-05 12:34:56.1205,1902-12-25 +1902-05-05 12:34:56.1206,1902-12-25 +1902-05-05 12:34:56.1207,1902-12-25 +1902-05-05 12:34:56.1208,1902-12-25 +1902-05-05 12:34:56.1209,1902-12-25 +1902-05-05 12:34:56.121,1902-12-25 +1902-05-05 12:34:56.1211,1902-12-25 +1902-05-05 12:34:56.1212,1902-12-25 +1902-05-05 12:34:56.1213,1902-12-25 +1902-05-05 12:34:56.1214,1902-12-25 +1902-05-05 12:34:56.1215,1902-12-25 +1902-05-05 12:34:56.1216,1902-12-25 +1902-05-05 12:34:56.1217,1902-12-25 +1902-05-05 12:34:56.1218,1902-12-25 +1902-05-05 12:34:56.1219,1902-12-25 +1902-05-05 12:34:56.122,1902-12-25 +1902-05-05 12:34:56.1221,1902-12-25 +1902-05-05 12:34:56.1222,1902-12-25 +1902-05-05 12:34:56.1223,1902-12-25 +1902-05-05 12:34:56.1224,1902-12-25 +1902-05-05 12:34:56.1225,1902-12-25 +1902-05-05 12:34:56.1226,1902-12-25 +1902-05-05 12:34:56.1227,1902-12-25 +1902-05-05 12:34:56.1228,1902-12-25 +1902-05-05 12:34:56.1229,1902-12-25 +1902-05-05 12:34:56.123,1902-12-25 +1902-05-05 12:34:56.1231,1902-12-25 +1902-05-05 12:34:56.1232,1902-12-25 +1902-05-05 12:34:56.1233,1902-12-25 +1902-05-05 12:34:56.1234,1902-12-25 +1902-05-05 12:34:56.1235,1902-12-25 +1902-05-05 12:34:56.1236,1902-12-25 +1902-05-05 12:34:56.1237,1902-12-25 +1902-05-05 12:34:56.1238,1902-12-25 +1902-05-05 12:34:56.1239,1902-12-25 +1902-05-05 12:34:56.124,1902-12-25 +1902-05-05 12:34:56.1241,1902-12-25 +1902-05-05 12:34:56.1242,1902-12-25 +1902-05-05 12:34:56.1243,1902-12-25 +1902-05-05 12:34:56.1244,1902-12-25 +1902-05-05 12:34:56.1245,1902-12-25 +1902-05-05 12:34:56.1246,1902-12-25 +1902-05-05 12:34:56.1247,1902-12-25 +1902-05-05 12:34:56.1248,1902-12-25 +1902-05-05 12:34:56.1249,1902-12-25 +1902-05-05 12:34:56.125,1902-12-25 +1902-05-05 12:34:56.1251,1902-12-25 +1902-05-05 12:34:56.1252,1902-12-25 +1902-05-05 12:34:56.1253,1902-12-25 +1902-05-05 12:34:56.1254,1902-12-25 +1902-05-05 12:34:56.1255,1902-12-25 +1902-05-05 12:34:56.1256,1902-12-25 +1902-05-05 12:34:56.1257,1902-12-25 +1902-05-05 12:34:56.1258,1902-12-25 +1902-05-05 12:34:56.1259,1902-12-25 +1902-05-05 12:34:56.126,1902-12-25 +1902-05-05 12:34:56.1261,1902-12-25 +1902-05-05 12:34:56.1262,1902-12-25 +1902-05-05 12:34:56.1263,1902-12-25 +1902-05-05 12:34:56.1264,1902-12-25 +1902-05-05 12:34:56.1265,1902-12-25 +1902-05-05 12:34:56.1266,1902-12-25 +1902-05-05 12:34:56.1267,1902-12-25 +1902-05-05 12:34:56.1268,1902-12-25 +1902-05-05 12:34:56.1269,1902-12-25 +1902-05-05 12:34:56.127,1902-12-25 +1902-05-05 12:34:56.1271,1902-12-25 +1902-05-05 12:34:56.1272,1902-12-25 +1902-05-05 12:34:56.1273,1902-12-25 +1902-05-05 12:34:56.1274,1902-12-25 +1902-05-05 12:34:56.1275,1902-12-25 +1902-05-05 12:34:56.1276,1902-12-25 +1902-05-05 12:34:56.1277,1902-12-25 +1902-05-05 12:34:56.1278,1902-12-25 +1902-05-05 12:34:56.1279,1902-12-25 +1902-05-05 12:34:56.128,1902-12-25 +1902-05-05 12:34:56.1281,1902-12-25 +1902-05-05 12:34:56.1282,1902-12-25 +1902-05-05 12:34:56.1283,1902-12-25 +1902-05-05 12:34:56.1284,1902-12-25 +1902-05-05 12:34:56.1285,1902-12-25 +1902-05-05 12:34:56.1286,1902-12-25 +1902-05-05 12:34:56.1287,1902-12-25 +1902-05-05 12:34:56.1288,1902-12-25 +1902-05-05 12:34:56.1289,1902-12-25 +1902-05-05 12:34:56.129,1902-12-25 +1902-05-05 12:34:56.1291,1902-12-25 +1902-05-05 12:34:56.1292,1902-12-25 +1902-05-05 12:34:56.1293,1902-12-25 +1902-05-05 12:34:56.1294,1902-12-25 +1902-05-05 12:34:56.1295,1902-12-25 +1902-05-05 12:34:56.1296,1902-12-25 +1902-05-05 12:34:56.1297,1902-12-25 +1902-05-05 12:34:56.1298,1902-12-25 +1902-05-05 12:34:56.1299,1902-12-25 +1902-05-05 12:34:56.13,1902-12-25 +1902-05-05 12:34:56.1301,1902-12-25 +1902-05-05 12:34:56.1302,1902-12-25 +1902-05-05 12:34:56.1303,1902-12-25 +1902-05-05 12:34:56.1304,1902-12-25 +1902-05-05 12:34:56.1305,1902-12-25 +1902-05-05 12:34:56.1306,1902-12-25 +1902-05-05 12:34:56.1307,1902-12-25 +1902-05-05 12:34:56.1308,1902-12-25 +1902-05-05 12:34:56.1309,1902-12-25 +1902-05-05 12:34:56.131,1902-12-25 +1902-05-05 12:34:56.1311,1902-12-25 +1902-05-05 12:34:56.1312,1902-12-25 +1902-05-05 12:34:56.1313,1902-12-25 +1902-05-05 12:34:56.1314,1902-12-25 +1902-05-05 12:34:56.1315,1902-12-25 +1902-05-05 12:34:56.1316,1902-12-25 +1902-05-05 12:34:56.1317,1902-12-25 +1902-05-05 12:34:56.1318,1902-12-25 +1902-05-05 12:34:56.1319,1902-12-25 +1902-05-05 12:34:56.132,1902-12-25 +1902-05-05 12:34:56.1321,1902-12-25 +1902-05-05 12:34:56.1322,1902-12-25 +1902-05-05 12:34:56.1323,1902-12-25 +1902-05-05 12:34:56.1324,1902-12-25 +1902-05-05 12:34:56.1325,1902-12-25 +1902-05-05 12:34:56.1326,1902-12-25 +1902-05-05 12:34:56.1327,1902-12-25 +1902-05-05 12:34:56.1328,1902-12-25 +1902-05-05 12:34:56.1329,1902-12-25 +1902-05-05 12:34:56.133,1902-12-25 +1902-05-05 12:34:56.1331,1902-12-25 +1902-05-05 12:34:56.1332,1902-12-25 +1902-05-05 12:34:56.1333,1902-12-25 +1902-05-05 12:34:56.1334,1902-12-25 +1902-05-05 12:34:56.1335,1902-12-25 +1902-05-05 12:34:56.1336,1902-12-25 +1902-05-05 12:34:56.1337,1902-12-25 +1902-05-05 12:34:56.1338,1902-12-25 +1902-05-05 12:34:56.1339,1902-12-25 +1902-05-05 12:34:56.134,1902-12-25 +1902-05-05 12:34:56.1341,1902-12-25 +1902-05-05 12:34:56.1342,1902-12-25 +1902-05-05 12:34:56.1343,1902-12-25 +1902-05-05 12:34:56.1344,1902-12-25 +1902-05-05 12:34:56.1345,1902-12-25 +1902-05-05 12:34:56.1346,1902-12-25 +1902-05-05 12:34:56.1347,1902-12-25 +1902-05-05 12:34:56.1348,1902-12-25 +1902-05-05 12:34:56.1349,1902-12-25 +1902-05-05 12:34:56.135,1902-12-25 +1902-05-05 12:34:56.1351,1902-12-25 +1902-05-05 12:34:56.1352,1902-12-25 +1902-05-05 12:34:56.1353,1902-12-25 +1902-05-05 12:34:56.1354,1902-12-25 +1902-05-05 12:34:56.1355,1902-12-25 +1902-05-05 12:34:56.1356,1902-12-25 +1902-05-05 12:34:56.1357,1902-12-25 +1902-05-05 12:34:56.1358,1902-12-25 +1902-05-05 12:34:56.1359,1902-12-25 +1902-05-05 12:34:56.136,1902-12-25 +1902-05-05 12:34:56.1361,1902-12-25 +1902-05-05 12:34:56.1362,1902-12-25 +1902-05-05 12:34:56.1363,1902-12-25 +1902-05-05 12:34:56.1364,1902-12-25 +1902-05-05 12:34:56.1365,1902-12-25 +1902-05-05 12:34:56.1366,1902-12-25 +1902-05-05 12:34:56.1367,1902-12-25 +1902-05-05 12:34:56.1368,1902-12-25 +1902-05-05 12:34:56.1369,1902-12-25 +1902-05-05 12:34:56.137,1902-12-25 +1902-05-05 12:34:56.1371,1902-12-25 +1902-05-05 12:34:56.1372,1902-12-25 +1902-05-05 12:34:56.1373,1902-12-25 +1902-05-05 12:34:56.1374,1902-12-25 +1902-05-05 12:34:56.1375,1902-12-25 +1902-05-05 12:34:56.1376,1902-12-25 +1902-05-05 12:34:56.1377,1902-12-25 +1902-05-05 12:34:56.1378,1902-12-25 +1902-05-05 12:34:56.1379,1902-12-25 +1902-05-05 12:34:56.138,1902-12-25 +1902-05-05 12:34:56.1381,1902-12-25 +1902-05-05 12:34:56.1382,1902-12-25 +1902-05-05 12:34:56.1383,1902-12-25 +1902-05-05 12:34:56.1384,1902-12-25 +1902-05-05 12:34:56.1385,1902-12-25 +1902-05-05 12:34:56.1386,1902-12-25 +1902-05-05 12:34:56.1387,1902-12-25 +1902-05-05 12:34:56.1388,1902-12-25 +1902-05-05 12:34:56.1389,1902-12-25 +1902-05-05 12:34:56.139,1902-12-25 +1902-05-05 12:34:56.1391,1902-12-25 +1902-05-05 12:34:56.1392,1902-12-25 +1902-05-05 12:34:56.1393,1902-12-25 +1902-05-05 12:34:56.1394,1902-12-25 +1902-05-05 12:34:56.1395,1902-12-25 +1902-05-05 12:34:56.1396,1902-12-25 +1902-05-05 12:34:56.1397,1902-12-25 +1902-05-05 12:34:56.1398,1902-12-25 +1902-05-05 12:34:56.1399,1902-12-25 +1902-05-05 12:34:56.14,1902-12-25 +1902-05-05 12:34:56.1401,1902-12-25 +1902-05-05 12:34:56.1402,1902-12-25 +1902-05-05 12:34:56.1403,1902-12-25 +1902-05-05 12:34:56.1404,1902-12-25 +1902-05-05 12:34:56.1405,1902-12-25 +1902-05-05 12:34:56.1406,1902-12-25 +1902-05-05 12:34:56.1407,1902-12-25 +1902-05-05 12:34:56.1408,1902-12-25 +1902-05-05 12:34:56.1409,1902-12-25 +1902-05-05 12:34:56.141,1902-12-25 +1902-05-05 12:34:56.1411,1902-12-25 +1902-05-05 12:34:56.1412,1902-12-25 +1902-05-05 12:34:56.1413,1902-12-25 +1902-05-05 12:34:56.1414,1902-12-25 +1902-05-05 12:34:56.1415,1902-12-25 +1902-05-05 12:34:56.1416,1902-12-25 +1902-05-05 12:34:56.1417,1902-12-25 +1902-05-05 12:34:56.1418,1902-12-25 +1902-05-05 12:34:56.1419,1902-12-25 +1902-05-05 12:34:56.142,1902-12-25 +1902-05-05 12:34:56.1421,1902-12-25 +1902-05-05 12:34:56.1422,1902-12-25 +1902-05-05 12:34:56.1423,1902-12-25 +1902-05-05 12:34:56.1424,1902-12-25 +1902-05-05 12:34:56.1425,1902-12-25 +1902-05-05 12:34:56.1426,1902-12-25 +1902-05-05 12:34:56.1427,1902-12-25 +1902-05-05 12:34:56.1428,1902-12-25 +1902-05-05 12:34:56.1429,1902-12-25 +1902-05-05 12:34:56.143,1902-12-25 +1902-05-05 12:34:56.1431,1902-12-25 +1902-05-05 12:34:56.1432,1902-12-25 +1902-05-05 12:34:56.1433,1902-12-25 +1902-05-05 12:34:56.1434,1902-12-25 +1902-05-05 12:34:56.1435,1902-12-25 +1902-05-05 12:34:56.1436,1902-12-25 +1902-05-05 12:34:56.1437,1902-12-25 +1902-05-05 12:34:56.1438,1902-12-25 +1902-05-05 12:34:56.1439,1902-12-25 +1902-05-05 12:34:56.144,1902-12-25 +1902-05-05 12:34:56.1441,1902-12-25 +1902-05-05 12:34:56.1442,1902-12-25 +1902-05-05 12:34:56.1443,1902-12-25 +1902-05-05 12:34:56.1444,1902-12-25 +1902-05-05 12:34:56.1445,1902-12-25 +1902-05-05 12:34:56.1446,1902-12-25 +1902-05-05 12:34:56.1447,1902-12-25 +1902-05-05 12:34:56.1448,1902-12-25 +1902-05-05 12:34:56.1449,1902-12-25 +1902-05-05 12:34:56.145,1902-12-25 +1902-05-05 12:34:56.1451,1902-12-25 +1902-05-05 12:34:56.1452,1902-12-25 +1902-05-05 12:34:56.1453,1902-12-25 +1902-05-05 12:34:56.1454,1902-12-25 +1902-05-05 12:34:56.1455,1902-12-25 +1902-05-05 12:34:56.1456,1902-12-25 +1902-05-05 12:34:56.1457,1902-12-25 +1902-05-05 12:34:56.1458,1902-12-25 +1902-05-05 12:34:56.1459,1902-12-25 +1902-05-05 12:34:56.146,1902-12-25 +1902-05-05 12:34:56.1461,1902-12-25 +1902-05-05 12:34:56.1462,1902-12-25 +1902-05-05 12:34:56.1463,1902-12-25 +1902-05-05 12:34:56.1464,1902-12-25 +1902-05-05 12:34:56.1465,1902-12-25 +1902-05-05 12:34:56.1466,1902-12-25 +1902-05-05 12:34:56.1467,1902-12-25 +1902-05-05 12:34:56.1468,1902-12-25 +1902-05-05 12:34:56.1469,1902-12-25 +1902-05-05 12:34:56.147,1902-12-25 +1902-05-05 12:34:56.1471,1902-12-25 +1902-05-05 12:34:56.1472,1902-12-25 +1902-05-05 12:34:56.1473,1902-12-25 +1902-05-05 12:34:56.1474,1902-12-25 +1902-05-05 12:34:56.1475,1902-12-25 +1902-05-05 12:34:56.1476,1902-12-25 +1902-05-05 12:34:56.1477,1902-12-25 +1902-05-05 12:34:56.1478,1902-12-25 +1902-05-05 12:34:56.1479,1902-12-25 +1902-05-05 12:34:56.148,1902-12-25 +1902-05-05 12:34:56.1481,1902-12-25 +1902-05-05 12:34:56.1482,1902-12-25 +1902-05-05 12:34:56.1483,1902-12-25 +1902-05-05 12:34:56.1484,1902-12-25 +1902-05-05 12:34:56.1485,1902-12-25 +1902-05-05 12:34:56.1486,1902-12-25 +1902-05-05 12:34:56.1487,1902-12-25 +1902-05-05 12:34:56.1488,1902-12-25 +1902-05-05 12:34:56.1489,1902-12-25 +1902-05-05 12:34:56.149,1902-12-25 +1902-05-05 12:34:56.1491,1902-12-25 +1902-05-05 12:34:56.1492,1902-12-25 +1902-05-05 12:34:56.1493,1902-12-25 +1902-05-05 12:34:56.1494,1902-12-25 +1902-05-05 12:34:56.1495,1902-12-25 +1902-05-05 12:34:56.1496,1902-12-25 +1902-05-05 12:34:56.1497,1902-12-25 +1902-05-05 12:34:56.1498,1902-12-25 +1902-05-05 12:34:56.1499,1902-12-25 +1902-05-05 12:34:56.15,1902-12-25 +1902-05-05 12:34:56.1501,1902-12-25 +1902-05-05 12:34:56.1502,1902-12-25 +1902-05-05 12:34:56.1503,1902-12-25 +1902-05-05 12:34:56.1504,1902-12-25 +1902-05-05 12:34:56.1505,1902-12-25 +1902-05-05 12:34:56.1506,1902-12-25 +1902-05-05 12:34:56.1507,1902-12-25 +1902-05-05 12:34:56.1508,1902-12-25 +1902-05-05 12:34:56.1509,1902-12-25 +1902-05-05 12:34:56.151,1902-12-25 +1902-05-05 12:34:56.1511,1902-12-25 +1902-05-05 12:34:56.1512,1902-12-25 +1902-05-05 12:34:56.1513,1902-12-25 +1902-05-05 12:34:56.1514,1902-12-25 +1902-05-05 12:34:56.1515,1902-12-25 +1902-05-05 12:34:56.1516,1902-12-25 +1902-05-05 12:34:56.1517,1902-12-25 +1902-05-05 12:34:56.1518,1902-12-25 +1902-05-05 12:34:56.1519,1902-12-25 +1902-05-05 12:34:56.152,1902-12-25 +1902-05-05 12:34:56.1521,1902-12-25 +1902-05-05 12:34:56.1522,1902-12-25 +1902-05-05 12:34:56.1523,1902-12-25 +1902-05-05 12:34:56.1524,1902-12-25 +1902-05-05 12:34:56.1525,1902-12-25 +1902-05-05 12:34:56.1526,1902-12-25 +1902-05-05 12:34:56.1527,1902-12-25 +1902-05-05 12:34:56.1528,1902-12-25 +1902-05-05 12:34:56.1529,1902-12-25 +1902-05-05 12:34:56.153,1902-12-25 +1902-05-05 12:34:56.1531,1902-12-25 +1902-05-05 12:34:56.1532,1902-12-25 +1902-05-05 12:34:56.1533,1902-12-25 +1902-05-05 12:34:56.1534,1902-12-25 +1902-05-05 12:34:56.1535,1902-12-25 +1902-05-05 12:34:56.1536,1902-12-25 +1902-05-05 12:34:56.1537,1902-12-25 +1902-05-05 12:34:56.1538,1902-12-25 +1902-05-05 12:34:56.1539,1902-12-25 +1902-05-05 12:34:56.154,1902-12-25 +1902-05-05 12:34:56.1541,1902-12-25 +1902-05-05 12:34:56.1542,1902-12-25 +1902-05-05 12:34:56.1543,1902-12-25 +1902-05-05 12:34:56.1544,1902-12-25 +1902-05-05 12:34:56.1545,1902-12-25 +1902-05-05 12:34:56.1546,1902-12-25 +1902-05-05 12:34:56.1547,1902-12-25 +1902-05-05 12:34:56.1548,1902-12-25 +1902-05-05 12:34:56.1549,1902-12-25 +1902-05-05 12:34:56.155,1902-12-25 +1902-05-05 12:34:56.1551,1902-12-25 +1902-05-05 12:34:56.1552,1902-12-25 +1902-05-05 12:34:56.1553,1902-12-25 +1902-05-05 12:34:56.1554,1902-12-25 +1902-05-05 12:34:56.1555,1902-12-25 +1902-05-05 12:34:56.1556,1902-12-25 +1902-05-05 12:34:56.1557,1902-12-25 +1902-05-05 12:34:56.1558,1902-12-25 +1902-05-05 12:34:56.1559,1902-12-25 +1902-05-05 12:34:56.156,1902-12-25 +1902-05-05 12:34:56.1561,1902-12-25 +1902-05-05 12:34:56.1562,1902-12-25 +1902-05-05 12:34:56.1563,1902-12-25 +1902-05-05 12:34:56.1564,1902-12-25 +1902-05-05 12:34:56.1565,1902-12-25 +1902-05-05 12:34:56.1566,1902-12-25 +1902-05-05 12:34:56.1567,1902-12-25 +1902-05-05 12:34:56.1568,1902-12-25 +1902-05-05 12:34:56.1569,1902-12-25 +1902-05-05 12:34:56.157,1902-12-25 +1902-05-05 12:34:56.1571,1902-12-25 +1902-05-05 12:34:56.1572,1902-12-25 +1902-05-05 12:34:56.1573,1902-12-25 +1902-05-05 12:34:56.1574,1902-12-25 +1902-05-05 12:34:56.1575,1902-12-25 +1902-05-05 12:34:56.1576,1902-12-25 +1902-05-05 12:34:56.1577,1902-12-25 +1902-05-05 12:34:56.1578,1902-12-25 +1902-05-05 12:34:56.1579,1902-12-25 +1902-05-05 12:34:56.158,1902-12-25 +1902-05-05 12:34:56.1581,1902-12-25 +1902-05-05 12:34:56.1582,1902-12-25 +1902-05-05 12:34:56.1583,1902-12-25 +1902-05-05 12:34:56.1584,1902-12-25 +1902-05-05 12:34:56.1585,1902-12-25 +1902-05-05 12:34:56.1586,1902-12-25 +1902-05-05 12:34:56.1587,1902-12-25 +1902-05-05 12:34:56.1588,1902-12-25 +1902-05-05 12:34:56.1589,1902-12-25 +1902-05-05 12:34:56.159,1902-12-25 +1902-05-05 12:34:56.1591,1902-12-25 +1902-05-05 12:34:56.1592,1902-12-25 +1902-05-05 12:34:56.1593,1902-12-25 +1902-05-05 12:34:56.1594,1902-12-25 +1902-05-05 12:34:56.1595,1902-12-25 +1902-05-05 12:34:56.1596,1902-12-25 +1902-05-05 12:34:56.1597,1902-12-25 +1902-05-05 12:34:56.1598,1902-12-25 +1902-05-05 12:34:56.1599,1902-12-25 +1902-05-05 12:34:56.16,1902-12-25 +1902-05-05 12:34:56.1601,1902-12-25 +1902-05-05 12:34:56.1602,1902-12-25 +1902-05-05 12:34:56.1603,1902-12-25 +1902-05-05 12:34:56.1604,1902-12-25 +1902-05-05 12:34:56.1605,1902-12-25 +1902-05-05 12:34:56.1606,1902-12-25 +1902-05-05 12:34:56.1607,1902-12-25 +1902-05-05 12:34:56.1608,1902-12-25 +1902-05-05 12:34:56.1609,1902-12-25 +1902-05-05 12:34:56.161,1902-12-25 +1902-05-05 12:34:56.1611,1902-12-25 +1902-05-05 12:34:56.1612,1902-12-25 +1902-05-05 12:34:56.1613,1902-12-25 +1902-05-05 12:34:56.1614,1902-12-25 +1902-05-05 12:34:56.1615,1902-12-25 +1902-05-05 12:34:56.1616,1902-12-25 +1902-05-05 12:34:56.1617,1902-12-25 +1902-05-05 12:34:56.1618,1902-12-25 +1902-05-05 12:34:56.1619,1902-12-25 +1902-05-05 12:34:56.162,1902-12-25 +1902-05-05 12:34:56.1621,1902-12-25 +1902-05-05 12:34:56.1622,1902-12-25 +1902-05-05 12:34:56.1623,1902-12-25 +1902-05-05 12:34:56.1624,1902-12-25 +1902-05-05 12:34:56.1625,1902-12-25 +1902-05-05 12:34:56.1626,1902-12-25 +1902-05-05 12:34:56.1627,1902-12-25 +1902-05-05 12:34:56.1628,1902-12-25 +1902-05-05 12:34:56.1629,1902-12-25 +1902-05-05 12:34:56.163,1902-12-25 +1902-05-05 12:34:56.1631,1902-12-25 +1902-05-05 12:34:56.1632,1902-12-25 +1902-05-05 12:34:56.1633,1902-12-25 +1902-05-05 12:34:56.1634,1902-12-25 +1902-05-05 12:34:56.1635,1902-12-25 +1902-05-05 12:34:56.1636,1902-12-25 +1902-05-05 12:34:56.1637,1902-12-25 +1902-05-05 12:34:56.1638,1902-12-25 +1902-05-05 12:34:56.1639,1902-12-25 +1902-05-05 12:34:56.164,1902-12-25 +1902-05-05 12:34:56.1641,1902-12-25 +1902-05-05 12:34:56.1642,1902-12-25 +1902-05-05 12:34:56.1643,1902-12-25 +1902-05-05 12:34:56.1644,1902-12-25 +1902-05-05 12:34:56.1645,1902-12-25 +1902-05-05 12:34:56.1646,1902-12-25 +1902-05-05 12:34:56.1647,1902-12-25 +1902-05-05 12:34:56.1648,1902-12-25 +1902-05-05 12:34:56.1649,1902-12-25 +1902-05-05 12:34:56.165,1902-12-25 +1902-05-05 12:34:56.1651,1902-12-25 +1902-05-05 12:34:56.1652,1902-12-25 +1902-05-05 12:34:56.1653,1902-12-25 +1902-05-05 12:34:56.1654,1902-12-25 +1902-05-05 12:34:56.1655,1902-12-25 +1902-05-05 12:34:56.1656,1902-12-25 +1902-05-05 12:34:56.1657,1902-12-25 +1902-05-05 12:34:56.1658,1902-12-25 +1902-05-05 12:34:56.1659,1902-12-25 +1902-05-05 12:34:56.166,1902-12-25 +1902-05-05 12:34:56.1661,1902-12-25 +1902-05-05 12:34:56.1662,1902-12-25 +1902-05-05 12:34:56.1663,1902-12-25 +1902-05-05 12:34:56.1664,1902-12-25 +1902-05-05 12:34:56.1665,1902-12-25 +1902-05-05 12:34:56.1666,1902-12-25 +1902-05-05 12:34:56.1667,1902-12-25 +1902-05-05 12:34:56.1668,1902-12-25 +1902-05-05 12:34:56.1669,1902-12-25 +1902-05-05 12:34:56.167,1902-12-25 +1902-05-05 12:34:56.1671,1902-12-25 +1902-05-05 12:34:56.1672,1902-12-25 +1902-05-05 12:34:56.1673,1902-12-25 +1902-05-05 12:34:56.1674,1902-12-25 +1902-05-05 12:34:56.1675,1902-12-25 +1902-05-05 12:34:56.1676,1902-12-25 +1902-05-05 12:34:56.1677,1902-12-25 +1902-05-05 12:34:56.1678,1902-12-25 +1902-05-05 12:34:56.1679,1902-12-25 +1902-05-05 12:34:56.168,1902-12-25 +1902-05-05 12:34:56.1681,1902-12-25 +1902-05-05 12:34:56.1682,1902-12-25 +1902-05-05 12:34:56.1683,1902-12-25 +1902-05-05 12:34:56.1684,1902-12-25 +1902-05-05 12:34:56.1685,1902-12-25 +1902-05-05 12:34:56.1686,1902-12-25 +1902-05-05 12:34:56.1687,1902-12-25 +1902-05-05 12:34:56.1688,1902-12-25 +1902-05-05 12:34:56.1689,1902-12-25 +1902-05-05 12:34:56.169,1902-12-25 +1902-05-05 12:34:56.1691,1902-12-25 +1902-05-05 12:34:56.1692,1902-12-25 +1902-05-05 12:34:56.1693,1902-12-25 +1902-05-05 12:34:56.1694,1902-12-25 +1902-05-05 12:34:56.1695,1902-12-25 +1902-05-05 12:34:56.1696,1902-12-25 +1902-05-05 12:34:56.1697,1902-12-25 +1902-05-05 12:34:56.1698,1902-12-25 +1902-05-05 12:34:56.1699,1902-12-25 +1902-05-05 12:34:56.17,1902-12-25 +1902-05-05 12:34:56.1701,1902-12-25 +1902-05-05 12:34:56.1702,1902-12-25 +1902-05-05 12:34:56.1703,1902-12-25 +1902-05-05 12:34:56.1704,1902-12-25 +1902-05-05 12:34:56.1705,1902-12-25 +1902-05-05 12:34:56.1706,1902-12-25 +1902-05-05 12:34:56.1707,1902-12-25 +1902-05-05 12:34:56.1708,1902-12-25 +1902-05-05 12:34:56.1709,1902-12-25 +1902-05-05 12:34:56.171,1902-12-25 +1902-05-05 12:34:56.1711,1902-12-25 +1902-05-05 12:34:56.1712,1902-12-25 +1902-05-05 12:34:56.1713,1902-12-25 +1902-05-05 12:34:56.1714,1902-12-25 +1902-05-05 12:34:56.1715,1902-12-25 +1902-05-05 12:34:56.1716,1902-12-25 +1902-05-05 12:34:56.1717,1902-12-25 +1902-05-05 12:34:56.1718,1902-12-25 +1902-05-05 12:34:56.1719,1902-12-25 +1902-05-05 12:34:56.172,1902-12-25 +1902-05-05 12:34:56.1721,1902-12-25 +1902-05-05 12:34:56.1722,1902-12-25 +1902-05-05 12:34:56.1723,1902-12-25 +1902-05-05 12:34:56.1724,1902-12-25 +1902-05-05 12:34:56.1725,1902-12-25 +1902-05-05 12:34:56.1726,1902-12-25 +1902-05-05 12:34:56.1727,1902-12-25 +1902-05-05 12:34:56.1728,1902-12-25 +1902-05-05 12:34:56.1729,1902-12-25 +1902-05-05 12:34:56.173,1902-12-25 +1902-05-05 12:34:56.1731,1902-12-25 +1902-05-05 12:34:56.1732,1902-12-25 +1902-05-05 12:34:56.1733,1902-12-25 +1902-05-05 12:34:56.1734,1902-12-25 +1902-05-05 12:34:56.1735,1902-12-25 +1902-05-05 12:34:56.1736,1902-12-25 +1902-05-05 12:34:56.1737,1902-12-25 +1902-05-05 12:34:56.1738,1902-12-25 +1902-05-05 12:34:56.1739,1902-12-25 +1902-05-05 12:34:56.174,1902-12-25 +1902-05-05 12:34:56.1741,1902-12-25 +1902-05-05 12:34:56.1742,1902-12-25 +1902-05-05 12:34:56.1743,1902-12-25 +1902-05-05 12:34:56.1744,1902-12-25 +1902-05-05 12:34:56.1745,1902-12-25 +1902-05-05 12:34:56.1746,1902-12-25 +1902-05-05 12:34:56.1747,1902-12-25 +1902-05-05 12:34:56.1748,1902-12-25 +1902-05-05 12:34:56.1749,1902-12-25 +1902-05-05 12:34:56.175,1902-12-25 +1902-05-05 12:34:56.1751,1902-12-25 +1902-05-05 12:34:56.1752,1902-12-25 +1902-05-05 12:34:56.1753,1902-12-25 +1902-05-05 12:34:56.1754,1902-12-25 +1902-05-05 12:34:56.1755,1902-12-25 +1902-05-05 12:34:56.1756,1902-12-25 +1902-05-05 12:34:56.1757,1902-12-25 +1902-05-05 12:34:56.1758,1902-12-25 +1902-05-05 12:34:56.1759,1902-12-25 +1902-05-05 12:34:56.176,1902-12-25 +1902-05-05 12:34:56.1761,1902-12-25 +1902-05-05 12:34:56.1762,1902-12-25 +1902-05-05 12:34:56.1763,1902-12-25 +1902-05-05 12:34:56.1764,1902-12-25 +1902-05-05 12:34:56.1765,1902-12-25 +1902-05-05 12:34:56.1766,1902-12-25 +1902-05-05 12:34:56.1767,1902-12-25 +1902-05-05 12:34:56.1768,1902-12-25 +1902-05-05 12:34:56.1769,1902-12-25 +1902-05-05 12:34:56.177,1902-12-25 +1902-05-05 12:34:56.1771,1902-12-25 +1902-05-05 12:34:56.1772,1902-12-25 +1902-05-05 12:34:56.1773,1902-12-25 +1902-05-05 12:34:56.1774,1902-12-25 +1902-05-05 12:34:56.1775,1902-12-25 +1902-05-05 12:34:56.1776,1902-12-25 +1902-05-05 12:34:56.1777,1902-12-25 +1902-05-05 12:34:56.1778,1902-12-25 +1902-05-05 12:34:56.1779,1902-12-25 +1902-05-05 12:34:56.178,1902-12-25 +1902-05-05 12:34:56.1781,1902-12-25 +1902-05-05 12:34:56.1782,1902-12-25 +1902-05-05 12:34:56.1783,1902-12-25 +1902-05-05 12:34:56.1784,1902-12-25 +1902-05-05 12:34:56.1785,1902-12-25 +1902-05-05 12:34:56.1786,1902-12-25 +1902-05-05 12:34:56.1787,1902-12-25 +1902-05-05 12:34:56.1788,1902-12-25 +1902-05-05 12:34:56.1789,1902-12-25 +1902-05-05 12:34:56.179,1902-12-25 +1902-05-05 12:34:56.1791,1902-12-25 +1902-05-05 12:34:56.1792,1902-12-25 +1902-05-05 12:34:56.1793,1902-12-25 +1902-05-05 12:34:56.1794,1902-12-25 +1902-05-05 12:34:56.1795,1902-12-25 +1902-05-05 12:34:56.1796,1902-12-25 +1902-05-05 12:34:56.1797,1902-12-25 +1902-05-05 12:34:56.1798,1902-12-25 +1902-05-05 12:34:56.1799,1902-12-25 +1902-05-05 12:34:56.18,1902-12-25 +1902-05-05 12:34:56.1801,1902-12-25 +1902-05-05 12:34:56.1802,1902-12-25 +1902-05-05 12:34:56.1803,1902-12-25 +1902-05-05 12:34:56.1804,1902-12-25 +1902-05-05 12:34:56.1805,1902-12-25 +1902-05-05 12:34:56.1806,1902-12-25 +1902-05-05 12:34:56.1807,1902-12-25 +1902-05-05 12:34:56.1808,1902-12-25 +1902-05-05 12:34:56.1809,1902-12-25 +1902-05-05 12:34:56.181,1902-12-25 +1902-05-05 12:34:56.1811,1902-12-25 +1902-05-05 12:34:56.1812,1902-12-25 +1902-05-05 12:34:56.1813,1902-12-25 +1902-05-05 12:34:56.1814,1902-12-25 +1902-05-05 12:34:56.1815,1902-12-25 +1902-05-05 12:34:56.1816,1902-12-25 +1902-05-05 12:34:56.1817,1902-12-25 +1902-05-05 12:34:56.1818,1902-12-25 +1902-05-05 12:34:56.1819,1902-12-25 +1902-05-05 12:34:56.182,1902-12-25 +1902-05-05 12:34:56.1821,1902-12-25 +1902-05-05 12:34:56.1822,1902-12-25 +1902-05-05 12:34:56.1823,1902-12-25 +1902-05-05 12:34:56.1824,1902-12-25 +1902-05-05 12:34:56.1825,1902-12-25 +1902-05-05 12:34:56.1826,1902-12-25 +1902-05-05 12:34:56.1827,1902-12-25 +1902-05-05 12:34:56.1828,1902-12-25 +1902-05-05 12:34:56.1829,1902-12-25 +1902-05-05 12:34:56.183,1902-12-25 +1902-05-05 12:34:56.1831,1902-12-25 +1902-05-05 12:34:56.1832,1902-12-25 +1902-05-05 12:34:56.1833,1902-12-25 +1902-05-05 12:34:56.1834,1902-12-25 +1902-05-05 12:34:56.1835,1902-12-25 +1902-05-05 12:34:56.1836,1902-12-25 +1902-05-05 12:34:56.1837,1902-12-25 +1902-05-05 12:34:56.1838,1902-12-25 +1902-05-05 12:34:56.1839,1902-12-25 +1902-05-05 12:34:56.184,1902-12-25 +1902-05-05 12:34:56.1841,1902-12-25 +1902-05-05 12:34:56.1842,1902-12-25 +1902-05-05 12:34:56.1843,1902-12-25 +1902-05-05 12:34:56.1844,1902-12-25 +1902-05-05 12:34:56.1845,1902-12-25 +1902-05-05 12:34:56.1846,1902-12-25 +1902-05-05 12:34:56.1847,1902-12-25 +1902-05-05 12:34:56.1848,1902-12-25 +1902-05-05 12:34:56.1849,1902-12-25 +1902-05-05 12:34:56.185,1902-12-25 +1902-05-05 12:34:56.1851,1902-12-25 +1902-05-05 12:34:56.1852,1902-12-25 +1902-05-05 12:34:56.1853,1902-12-25 +1902-05-05 12:34:56.1854,1902-12-25 +1902-05-05 12:34:56.1855,1902-12-25 +1902-05-05 12:34:56.1856,1902-12-25 +1902-05-05 12:34:56.1857,1902-12-25 +1902-05-05 12:34:56.1858,1902-12-25 +1902-05-05 12:34:56.1859,1902-12-25 +1902-05-05 12:34:56.186,1902-12-25 +1902-05-05 12:34:56.1861,1902-12-25 +1902-05-05 12:34:56.1862,1902-12-25 +1902-05-05 12:34:56.1863,1902-12-25 +1902-05-05 12:34:56.1864,1902-12-25 +1902-05-05 12:34:56.1865,1902-12-25 +1902-05-05 12:34:56.1866,1902-12-25 +1902-05-05 12:34:56.1867,1902-12-25 +1902-05-05 12:34:56.1868,1902-12-25 +1902-05-05 12:34:56.1869,1902-12-25 +1902-05-05 12:34:56.187,1902-12-25 +1902-05-05 12:34:56.1871,1902-12-25 +1902-05-05 12:34:56.1872,1902-12-25 +1902-05-05 12:34:56.1873,1902-12-25 +1902-05-05 12:34:56.1874,1902-12-25 +1902-05-05 12:34:56.1875,1902-12-25 +1902-05-05 12:34:56.1876,1902-12-25 +1902-05-05 12:34:56.1877,1902-12-25 +1902-05-05 12:34:56.1878,1902-12-25 +1902-05-05 12:34:56.1879,1902-12-25 +1902-05-05 12:34:56.188,1902-12-25 +1902-05-05 12:34:56.1881,1902-12-25 +1902-05-05 12:34:56.1882,1902-12-25 +1902-05-05 12:34:56.1883,1902-12-25 +1902-05-05 12:34:56.1884,1902-12-25 +1902-05-05 12:34:56.1885,1902-12-25 +1902-05-05 12:34:56.1886,1902-12-25 +1902-05-05 12:34:56.1887,1902-12-25 +1902-05-05 12:34:56.1888,1902-12-25 +1902-05-05 12:34:56.1889,1902-12-25 +1902-05-05 12:34:56.189,1902-12-25 +1902-05-05 12:34:56.1891,1902-12-25 +1902-05-05 12:34:56.1892,1902-12-25 +1902-05-05 12:34:56.1893,1902-12-25 +1902-05-05 12:34:56.1894,1902-12-25 +1902-05-05 12:34:56.1895,1902-12-25 +1902-05-05 12:34:56.1896,1902-12-25 +1902-05-05 12:34:56.1897,1902-12-25 +1902-05-05 12:34:56.1898,1902-12-25 +1902-05-05 12:34:56.1899,1902-12-25 +1902-05-05 12:34:56.19,1902-12-25 +1902-05-05 12:34:56.1901,1902-12-25 +1902-05-05 12:34:56.1902,1902-12-25 +1902-05-05 12:34:56.1903,1902-12-25 +1902-05-05 12:34:56.1904,1902-12-25 +1902-05-05 12:34:56.1905,1902-12-25 +1902-05-05 12:34:56.1906,1902-12-25 +1902-05-05 12:34:56.1907,1902-12-25 +1902-05-05 12:34:56.1908,1902-12-25 +1902-05-05 12:34:56.1909,1902-12-25 +1902-05-05 12:34:56.191,1902-12-25 +1902-05-05 12:34:56.1911,1902-12-25 +1902-05-05 12:34:56.1912,1902-12-25 +1902-05-05 12:34:56.1913,1902-12-25 +1902-05-05 12:34:56.1914,1902-12-25 +1902-05-05 12:34:56.1915,1902-12-25 +1902-05-05 12:34:56.1916,1902-12-25 +1902-05-05 12:34:56.1917,1902-12-25 +1902-05-05 12:34:56.1918,1902-12-25 +1902-05-05 12:34:56.1919,1902-12-25 +1902-05-05 12:34:56.192,1902-12-25 +1902-05-05 12:34:56.1921,1902-12-25 +1902-05-05 12:34:56.1922,1902-12-25 +1902-05-05 12:34:56.1923,1902-12-25 +1902-05-05 12:34:56.1924,1902-12-25 +1902-05-05 12:34:56.1925,1902-12-25 +1902-05-05 12:34:56.1926,1902-12-25 +1902-05-05 12:34:56.1927,1902-12-25 +1902-05-05 12:34:56.1928,1902-12-25 +1902-05-05 12:34:56.1929,1902-12-25 +1902-05-05 12:34:56.193,1902-12-25 +1902-05-05 12:34:56.1931,1902-12-25 +1902-05-05 12:34:56.1932,1902-12-25 +1902-05-05 12:34:56.1933,1902-12-25 +1902-05-05 12:34:56.1934,1902-12-25 +1902-05-05 12:34:56.1935,1902-12-25 +1902-05-05 12:34:56.1936,1902-12-25 +1902-05-05 12:34:56.1937,1902-12-25 +1902-05-05 12:34:56.1938,1902-12-25 +1902-05-05 12:34:56.1939,1902-12-25 +1902-05-05 12:34:56.194,1902-12-25 +1902-05-05 12:34:56.1941,1902-12-25 +1902-05-05 12:34:56.1942,1902-12-25 +1902-05-05 12:34:56.1943,1902-12-25 +1902-05-05 12:34:56.1944,1902-12-25 +1902-05-05 12:34:56.1945,1902-12-25 +1902-05-05 12:34:56.1946,1902-12-25 +1902-05-05 12:34:56.1947,1902-12-25 +1902-05-05 12:34:56.1948,1902-12-25 +1902-05-05 12:34:56.1949,1902-12-25 +1902-05-05 12:34:56.195,1902-12-25 +1902-05-05 12:34:56.1951,1902-12-25 +1902-05-05 12:34:56.1952,1902-12-25 +1902-05-05 12:34:56.1953,1902-12-25 +1902-05-05 12:34:56.1954,1902-12-25 +1902-05-05 12:34:56.1955,1902-12-25 +1902-05-05 12:34:56.1956,1902-12-25 +1902-05-05 12:34:56.1957,1902-12-25 +1902-05-05 12:34:56.1958,1902-12-25 +1902-05-05 12:34:56.1959,1902-12-25 +1902-05-05 12:34:56.196,1902-12-25 +1902-05-05 12:34:56.1961,1902-12-25 +1902-05-05 12:34:56.1962,1902-12-25 +1902-05-05 12:34:56.1963,1902-12-25 +1902-05-05 12:34:56.1964,1902-12-25 +1902-05-05 12:34:56.1965,1902-12-25 +1902-05-05 12:34:56.1966,1902-12-25 +1902-05-05 12:34:56.1967,1902-12-25 +1902-05-05 12:34:56.1968,1902-12-25 +1902-05-05 12:34:56.1969,1902-12-25 +1902-05-05 12:34:56.197,1902-12-25 +1902-05-05 12:34:56.1971,1902-12-25 +1902-05-05 12:34:56.1972,1902-12-25 +1902-05-05 12:34:56.1973,1902-12-25 +1902-05-05 12:34:56.1974,1902-12-25 +1902-05-05 12:34:56.1975,1902-12-25 +1902-05-05 12:34:56.1976,1902-12-25 +1902-05-05 12:34:56.1977,1902-12-25 +1902-05-05 12:34:56.1978,1902-12-25 +1902-05-05 12:34:56.1979,1902-12-25 +1902-05-05 12:34:56.198,1902-12-25 +1902-05-05 12:34:56.1981,1902-12-25 +1902-05-05 12:34:56.1982,1902-12-25 +1902-05-05 12:34:56.1983,1902-12-25 +1902-05-05 12:34:56.1984,1902-12-25 +1902-05-05 12:34:56.1985,1902-12-25 +1902-05-05 12:34:56.1986,1902-12-25 +1902-05-05 12:34:56.1987,1902-12-25 +1902-05-05 12:34:56.1988,1902-12-25 +1902-05-05 12:34:56.1989,1902-12-25 +1902-05-05 12:34:56.199,1902-12-25 +1902-05-05 12:34:56.1991,1902-12-25 +1902-05-05 12:34:56.1992,1902-12-25 +1902-05-05 12:34:56.1993,1902-12-25 +1902-05-05 12:34:56.1994,1902-12-25 +1902-05-05 12:34:56.1995,1902-12-25 +1902-05-05 12:34:56.1996,1902-12-25 +1902-05-05 12:34:56.1997,1902-12-25 +1902-05-05 12:34:56.1998,1902-12-25 +1902-05-05 12:34:56.1999,1902-12-25 +1903-05-05 12:34:56.1,1903-12-25 +1903-05-05 12:34:56.1001,1903-12-25 +1903-05-05 12:34:56.1002,1903-12-25 +1903-05-05 12:34:56.1003,1903-12-25 +1903-05-05 12:34:56.1004,1903-12-25 +1903-05-05 12:34:56.1005,1903-12-25 +1903-05-05 12:34:56.1006,1903-12-25 +1903-05-05 12:34:56.1007,1903-12-25 +1903-05-05 12:34:56.1008,1903-12-25 +1903-05-05 12:34:56.1009,1903-12-25 +1903-05-05 12:34:56.101,1903-12-25 +1903-05-05 12:34:56.1011,1903-12-25 +1903-05-05 12:34:56.1012,1903-12-25 +1903-05-05 12:34:56.1013,1903-12-25 +1903-05-05 12:34:56.1014,1903-12-25 +1903-05-05 12:34:56.1015,1903-12-25 +1903-05-05 12:34:56.1016,1903-12-25 +1903-05-05 12:34:56.1017,1903-12-25 +1903-05-05 12:34:56.1018,1903-12-25 +1903-05-05 12:34:56.1019,1903-12-25 +1903-05-05 12:34:56.102,1903-12-25 +1903-05-05 12:34:56.1021,1903-12-25 +1903-05-05 12:34:56.1022,1903-12-25 +1903-05-05 12:34:56.1023,1903-12-25 +1903-05-05 12:34:56.1024,1903-12-25 +1903-05-05 12:34:56.1025,1903-12-25 +1903-05-05 12:34:56.1026,1903-12-25 +1903-05-05 12:34:56.1027,1903-12-25 +1903-05-05 12:34:56.1028,1903-12-25 +1903-05-05 12:34:56.1029,1903-12-25 +1903-05-05 12:34:56.103,1903-12-25 +1903-05-05 12:34:56.1031,1903-12-25 +1903-05-05 12:34:56.1032,1903-12-25 +1903-05-05 12:34:56.1033,1903-12-25 +1903-05-05 12:34:56.1034,1903-12-25 +1903-05-05 12:34:56.1035,1903-12-25 +1903-05-05 12:34:56.1036,1903-12-25 +1903-05-05 12:34:56.1037,1903-12-25 +1903-05-05 12:34:56.1038,1903-12-25 +1903-05-05 12:34:56.1039,1903-12-25 +1903-05-05 12:34:56.104,1903-12-25 +1903-05-05 12:34:56.1041,1903-12-25 +1903-05-05 12:34:56.1042,1903-12-25 +1903-05-05 12:34:56.1043,1903-12-25 +1903-05-05 12:34:56.1044,1903-12-25 +1903-05-05 12:34:56.1045,1903-12-25 +1903-05-05 12:34:56.1046,1903-12-25 +1903-05-05 12:34:56.1047,1903-12-25 +1903-05-05 12:34:56.1048,1903-12-25 +1903-05-05 12:34:56.1049,1903-12-25 +1903-05-05 12:34:56.105,1903-12-25 +1903-05-05 12:34:56.1051,1903-12-25 +1903-05-05 12:34:56.1052,1903-12-25 +1903-05-05 12:34:56.1053,1903-12-25 +1903-05-05 12:34:56.1054,1903-12-25 +1903-05-05 12:34:56.1055,1903-12-25 +1903-05-05 12:34:56.1056,1903-12-25 +1903-05-05 12:34:56.1057,1903-12-25 +1903-05-05 12:34:56.1058,1903-12-25 +1903-05-05 12:34:56.1059,1903-12-25 +1903-05-05 12:34:56.106,1903-12-25 +1903-05-05 12:34:56.1061,1903-12-25 +1903-05-05 12:34:56.1062,1903-12-25 +1903-05-05 12:34:56.1063,1903-12-25 +1903-05-05 12:34:56.1064,1903-12-25 +1903-05-05 12:34:56.1065,1903-12-25 +1903-05-05 12:34:56.1066,1903-12-25 +1903-05-05 12:34:56.1067,1903-12-25 +1903-05-05 12:34:56.1068,1903-12-25 +1903-05-05 12:34:56.1069,1903-12-25 +1903-05-05 12:34:56.107,1903-12-25 +1903-05-05 12:34:56.1071,1903-12-25 +1903-05-05 12:34:56.1072,1903-12-25 +1903-05-05 12:34:56.1073,1903-12-25 +1903-05-05 12:34:56.1074,1903-12-25 +1903-05-05 12:34:56.1075,1903-12-25 +1903-05-05 12:34:56.1076,1903-12-25 +1903-05-05 12:34:56.1077,1903-12-25 +1903-05-05 12:34:56.1078,1903-12-25 +1903-05-05 12:34:56.1079,1903-12-25 +1903-05-05 12:34:56.108,1903-12-25 +1903-05-05 12:34:56.1081,1903-12-25 +1903-05-05 12:34:56.1082,1903-12-25 +1903-05-05 12:34:56.1083,1903-12-25 +1903-05-05 12:34:56.1084,1903-12-25 +1903-05-05 12:34:56.1085,1903-12-25 +1903-05-05 12:34:56.1086,1903-12-25 +1903-05-05 12:34:56.1087,1903-12-25 +1903-05-05 12:34:56.1088,1903-12-25 +1903-05-05 12:34:56.1089,1903-12-25 +1903-05-05 12:34:56.109,1903-12-25 +1903-05-05 12:34:56.1091,1903-12-25 +1903-05-05 12:34:56.1092,1903-12-25 +1903-05-05 12:34:56.1093,1903-12-25 +1903-05-05 12:34:56.1094,1903-12-25 +1903-05-05 12:34:56.1095,1903-12-25 +1903-05-05 12:34:56.1096,1903-12-25 +1903-05-05 12:34:56.1097,1903-12-25 +1903-05-05 12:34:56.1098,1903-12-25 +1903-05-05 12:34:56.1099,1903-12-25 +1903-05-05 12:34:56.11,1903-12-25 +1903-05-05 12:34:56.1101,1903-12-25 +1903-05-05 12:34:56.1102,1903-12-25 +1903-05-05 12:34:56.1103,1903-12-25 +1903-05-05 12:34:56.1104,1903-12-25 +1903-05-05 12:34:56.1105,1903-12-25 +1903-05-05 12:34:56.1106,1903-12-25 +1903-05-05 12:34:56.1107,1903-12-25 +1903-05-05 12:34:56.1108,1903-12-25 +1903-05-05 12:34:56.1109,1903-12-25 +1903-05-05 12:34:56.111,1903-12-25 +1903-05-05 12:34:56.1111,1903-12-25 +1903-05-05 12:34:56.1112,1903-12-25 +1903-05-05 12:34:56.1113,1903-12-25 +1903-05-05 12:34:56.1114,1903-12-25 +1903-05-05 12:34:56.1115,1903-12-25 +1903-05-05 12:34:56.1116,1903-12-25 +1903-05-05 12:34:56.1117,1903-12-25 +1903-05-05 12:34:56.1118,1903-12-25 +1903-05-05 12:34:56.1119,1903-12-25 +1903-05-05 12:34:56.112,1903-12-25 +1903-05-05 12:34:56.1121,1903-12-25 +1903-05-05 12:34:56.1122,1903-12-25 +1903-05-05 12:34:56.1123,1903-12-25 +1903-05-05 12:34:56.1124,1903-12-25 +1903-05-05 12:34:56.1125,1903-12-25 +1903-05-05 12:34:56.1126,1903-12-25 +1903-05-05 12:34:56.1127,1903-12-25 +1903-05-05 12:34:56.1128,1903-12-25 +1903-05-05 12:34:56.1129,1903-12-25 +1903-05-05 12:34:56.113,1903-12-25 +1903-05-05 12:34:56.1131,1903-12-25 +1903-05-05 12:34:56.1132,1903-12-25 +1903-05-05 12:34:56.1133,1903-12-25 +1903-05-05 12:34:56.1134,1903-12-25 +1903-05-05 12:34:56.1135,1903-12-25 +1903-05-05 12:34:56.1136,1903-12-25 +1903-05-05 12:34:56.1137,1903-12-25 +1903-05-05 12:34:56.1138,1903-12-25 +1903-05-05 12:34:56.1139,1903-12-25 +1903-05-05 12:34:56.114,1903-12-25 +1903-05-05 12:34:56.1141,1903-12-25 +1903-05-05 12:34:56.1142,1903-12-25 +1903-05-05 12:34:56.1143,1903-12-25 +1903-05-05 12:34:56.1144,1903-12-25 +1903-05-05 12:34:56.1145,1903-12-25 +1903-05-05 12:34:56.1146,1903-12-25 +1903-05-05 12:34:56.1147,1903-12-25 +1903-05-05 12:34:56.1148,1903-12-25 +1903-05-05 12:34:56.1149,1903-12-25 +1903-05-05 12:34:56.115,1903-12-25 +1903-05-05 12:34:56.1151,1903-12-25 +1903-05-05 12:34:56.1152,1903-12-25 +1903-05-05 12:34:56.1153,1903-12-25 +1903-05-05 12:34:56.1154,1903-12-25 +1903-05-05 12:34:56.1155,1903-12-25 +1903-05-05 12:34:56.1156,1903-12-25 +1903-05-05 12:34:56.1157,1903-12-25 +1903-05-05 12:34:56.1158,1903-12-25 +1903-05-05 12:34:56.1159,1903-12-25 +1903-05-05 12:34:56.116,1903-12-25 +1903-05-05 12:34:56.1161,1903-12-25 +1903-05-05 12:34:56.1162,1903-12-25 +1903-05-05 12:34:56.1163,1903-12-25 +1903-05-05 12:34:56.1164,1903-12-25 +1903-05-05 12:34:56.1165,1903-12-25 +1903-05-05 12:34:56.1166,1903-12-25 +1903-05-05 12:34:56.1167,1903-12-25 +1903-05-05 12:34:56.1168,1903-12-25 +1903-05-05 12:34:56.1169,1903-12-25 +1903-05-05 12:34:56.117,1903-12-25 +1903-05-05 12:34:56.1171,1903-12-25 +1903-05-05 12:34:56.1172,1903-12-25 +1903-05-05 12:34:56.1173,1903-12-25 +1903-05-05 12:34:56.1174,1903-12-25 +1903-05-05 12:34:56.1175,1903-12-25 +1903-05-05 12:34:56.1176,1903-12-25 +1903-05-05 12:34:56.1177,1903-12-25 +1903-05-05 12:34:56.1178,1903-12-25 +1903-05-05 12:34:56.1179,1903-12-25 +1903-05-05 12:34:56.118,1903-12-25 +1903-05-05 12:34:56.1181,1903-12-25 +1903-05-05 12:34:56.1182,1903-12-25 +1903-05-05 12:34:56.1183,1903-12-25 +1903-05-05 12:34:56.1184,1903-12-25 +1903-05-05 12:34:56.1185,1903-12-25 +1903-05-05 12:34:56.1186,1903-12-25 +1903-05-05 12:34:56.1187,1903-12-25 +1903-05-05 12:34:56.1188,1903-12-25 +1903-05-05 12:34:56.1189,1903-12-25 +1903-05-05 12:34:56.119,1903-12-25 +1903-05-05 12:34:56.1191,1903-12-25 +1903-05-05 12:34:56.1192,1903-12-25 +1903-05-05 12:34:56.1193,1903-12-25 +1903-05-05 12:34:56.1194,1903-12-25 +1903-05-05 12:34:56.1195,1903-12-25 +1903-05-05 12:34:56.1196,1903-12-25 +1903-05-05 12:34:56.1197,1903-12-25 +1903-05-05 12:34:56.1198,1903-12-25 +1903-05-05 12:34:56.1199,1903-12-25 +1903-05-05 12:34:56.12,1903-12-25 +1903-05-05 12:34:56.1201,1903-12-25 +1903-05-05 12:34:56.1202,1903-12-25 +1903-05-05 12:34:56.1203,1903-12-25 +1903-05-05 12:34:56.1204,1903-12-25 +1903-05-05 12:34:56.1205,1903-12-25 +1903-05-05 12:34:56.1206,1903-12-25 +1903-05-05 12:34:56.1207,1903-12-25 +1903-05-05 12:34:56.1208,1903-12-25 +1903-05-05 12:34:56.1209,1903-12-25 +1903-05-05 12:34:56.121,1903-12-25 +1903-05-05 12:34:56.1211,1903-12-25 +1903-05-05 12:34:56.1212,1903-12-25 +1903-05-05 12:34:56.1213,1903-12-25 +1903-05-05 12:34:56.1214,1903-12-25 +1903-05-05 12:34:56.1215,1903-12-25 +1903-05-05 12:34:56.1216,1903-12-25 +1903-05-05 12:34:56.1217,1903-12-25 +1903-05-05 12:34:56.1218,1903-12-25 +1903-05-05 12:34:56.1219,1903-12-25 +1903-05-05 12:34:56.122,1903-12-25 +1903-05-05 12:34:56.1221,1903-12-25 +1903-05-05 12:34:56.1222,1903-12-25 +1903-05-05 12:34:56.1223,1903-12-25 +1903-05-05 12:34:56.1224,1903-12-25 +1903-05-05 12:34:56.1225,1903-12-25 +1903-05-05 12:34:56.1226,1903-12-25 +1903-05-05 12:34:56.1227,1903-12-25 +1903-05-05 12:34:56.1228,1903-12-25 +1903-05-05 12:34:56.1229,1903-12-25 +1903-05-05 12:34:56.123,1903-12-25 +1903-05-05 12:34:56.1231,1903-12-25 +1903-05-05 12:34:56.1232,1903-12-25 +1903-05-05 12:34:56.1233,1903-12-25 +1903-05-05 12:34:56.1234,1903-12-25 +1903-05-05 12:34:56.1235,1903-12-25 +1903-05-05 12:34:56.1236,1903-12-25 +1903-05-05 12:34:56.1237,1903-12-25 +1903-05-05 12:34:56.1238,1903-12-25 +1903-05-05 12:34:56.1239,1903-12-25 +1903-05-05 12:34:56.124,1903-12-25 +1903-05-05 12:34:56.1241,1903-12-25 +1903-05-05 12:34:56.1242,1903-12-25 +1903-05-05 12:34:56.1243,1903-12-25 +1903-05-05 12:34:56.1244,1903-12-25 +1903-05-05 12:34:56.1245,1903-12-25 +1903-05-05 12:34:56.1246,1903-12-25 +1903-05-05 12:34:56.1247,1903-12-25 +1903-05-05 12:34:56.1248,1903-12-25 +1903-05-05 12:34:56.1249,1903-12-25 +1903-05-05 12:34:56.125,1903-12-25 +1903-05-05 12:34:56.1251,1903-12-25 +1903-05-05 12:34:56.1252,1903-12-25 +1903-05-05 12:34:56.1253,1903-12-25 +1903-05-05 12:34:56.1254,1903-12-25 +1903-05-05 12:34:56.1255,1903-12-25 +1903-05-05 12:34:56.1256,1903-12-25 +1903-05-05 12:34:56.1257,1903-12-25 +1903-05-05 12:34:56.1258,1903-12-25 +1903-05-05 12:34:56.1259,1903-12-25 +1903-05-05 12:34:56.126,1903-12-25 +1903-05-05 12:34:56.1261,1903-12-25 +1903-05-05 12:34:56.1262,1903-12-25 +1903-05-05 12:34:56.1263,1903-12-25 +1903-05-05 12:34:56.1264,1903-12-25 +1903-05-05 12:34:56.1265,1903-12-25 +1903-05-05 12:34:56.1266,1903-12-25 +1903-05-05 12:34:56.1267,1903-12-25 +1903-05-05 12:34:56.1268,1903-12-25 +1903-05-05 12:34:56.1269,1903-12-25 +1903-05-05 12:34:56.127,1903-12-25 +1903-05-05 12:34:56.1271,1903-12-25 +1903-05-05 12:34:56.1272,1903-12-25 +1903-05-05 12:34:56.1273,1903-12-25 +1903-05-05 12:34:56.1274,1903-12-25 +1903-05-05 12:34:56.1275,1903-12-25 +1903-05-05 12:34:56.1276,1903-12-25 +1903-05-05 12:34:56.1277,1903-12-25 +1903-05-05 12:34:56.1278,1903-12-25 +1903-05-05 12:34:56.1279,1903-12-25 +1903-05-05 12:34:56.128,1903-12-25 +1903-05-05 12:34:56.1281,1903-12-25 +1903-05-05 12:34:56.1282,1903-12-25 +1903-05-05 12:34:56.1283,1903-12-25 +1903-05-05 12:34:56.1284,1903-12-25 +1903-05-05 12:34:56.1285,1903-12-25 +1903-05-05 12:34:56.1286,1903-12-25 +1903-05-05 12:34:56.1287,1903-12-25 +1903-05-05 12:34:56.1288,1903-12-25 +1903-05-05 12:34:56.1289,1903-12-25 +1903-05-05 12:34:56.129,1903-12-25 +1903-05-05 12:34:56.1291,1903-12-25 +1903-05-05 12:34:56.1292,1903-12-25 +1903-05-05 12:34:56.1293,1903-12-25 +1903-05-05 12:34:56.1294,1903-12-25 +1903-05-05 12:34:56.1295,1903-12-25 +1903-05-05 12:34:56.1296,1903-12-25 +1903-05-05 12:34:56.1297,1903-12-25 +1903-05-05 12:34:56.1298,1903-12-25 +1903-05-05 12:34:56.1299,1903-12-25 +1903-05-05 12:34:56.13,1903-12-25 +1903-05-05 12:34:56.1301,1903-12-25 +1903-05-05 12:34:56.1302,1903-12-25 +1903-05-05 12:34:56.1303,1903-12-25 +1903-05-05 12:34:56.1304,1903-12-25 +1903-05-05 12:34:56.1305,1903-12-25 +1903-05-05 12:34:56.1306,1903-12-25 +1903-05-05 12:34:56.1307,1903-12-25 +1903-05-05 12:34:56.1308,1903-12-25 +1903-05-05 12:34:56.1309,1903-12-25 +1903-05-05 12:34:56.131,1903-12-25 +1903-05-05 12:34:56.1311,1903-12-25 +1903-05-05 12:34:56.1312,1903-12-25 +1903-05-05 12:34:56.1313,1903-12-25 +1903-05-05 12:34:56.1314,1903-12-25 +1903-05-05 12:34:56.1315,1903-12-25 +1903-05-05 12:34:56.1316,1903-12-25 +1903-05-05 12:34:56.1317,1903-12-25 +1903-05-05 12:34:56.1318,1903-12-25 +1903-05-05 12:34:56.1319,1903-12-25 +1903-05-05 12:34:56.132,1903-12-25 +1903-05-05 12:34:56.1321,1903-12-25 +1903-05-05 12:34:56.1322,1903-12-25 +1903-05-05 12:34:56.1323,1903-12-25 +1903-05-05 12:34:56.1324,1903-12-25 +1903-05-05 12:34:56.1325,1903-12-25 +1903-05-05 12:34:56.1326,1903-12-25 +1903-05-05 12:34:56.1327,1903-12-25 +1903-05-05 12:34:56.1328,1903-12-25 +1903-05-05 12:34:56.1329,1903-12-25 +1903-05-05 12:34:56.133,1903-12-25 +1903-05-05 12:34:56.1331,1903-12-25 +1903-05-05 12:34:56.1332,1903-12-25 +1903-05-05 12:34:56.1333,1903-12-25 +1903-05-05 12:34:56.1334,1903-12-25 +1903-05-05 12:34:56.1335,1903-12-25 +1903-05-05 12:34:56.1336,1903-12-25 +1903-05-05 12:34:56.1337,1903-12-25 +1903-05-05 12:34:56.1338,1903-12-25 +1903-05-05 12:34:56.1339,1903-12-25 +1903-05-05 12:34:56.134,1903-12-25 +1903-05-05 12:34:56.1341,1903-12-25 +1903-05-05 12:34:56.1342,1903-12-25 +1903-05-05 12:34:56.1343,1903-12-25 +1903-05-05 12:34:56.1344,1903-12-25 +1903-05-05 12:34:56.1345,1903-12-25 +1903-05-05 12:34:56.1346,1903-12-25 +1903-05-05 12:34:56.1347,1903-12-25 +1903-05-05 12:34:56.1348,1903-12-25 +1903-05-05 12:34:56.1349,1903-12-25 +1903-05-05 12:34:56.135,1903-12-25 +1903-05-05 12:34:56.1351,1903-12-25 +1903-05-05 12:34:56.1352,1903-12-25 +1903-05-05 12:34:56.1353,1903-12-25 +1903-05-05 12:34:56.1354,1903-12-25 +1903-05-05 12:34:56.1355,1903-12-25 +1903-05-05 12:34:56.1356,1903-12-25 +1903-05-05 12:34:56.1357,1903-12-25 +1903-05-05 12:34:56.1358,1903-12-25 +1903-05-05 12:34:56.1359,1903-12-25 +1903-05-05 12:34:56.136,1903-12-25 +1903-05-05 12:34:56.1361,1903-12-25 +1903-05-05 12:34:56.1362,1903-12-25 +1903-05-05 12:34:56.1363,1903-12-25 +1903-05-05 12:34:56.1364,1903-12-25 +1903-05-05 12:34:56.1365,1903-12-25 +1903-05-05 12:34:56.1366,1903-12-25 +1903-05-05 12:34:56.1367,1903-12-25 +1903-05-05 12:34:56.1368,1903-12-25 +1903-05-05 12:34:56.1369,1903-12-25 +1903-05-05 12:34:56.137,1903-12-25 +1903-05-05 12:34:56.1371,1903-12-25 +1903-05-05 12:34:56.1372,1903-12-25 +1903-05-05 12:34:56.1373,1903-12-25 +1903-05-05 12:34:56.1374,1903-12-25 +1903-05-05 12:34:56.1375,1903-12-25 +1903-05-05 12:34:56.1376,1903-12-25 +1903-05-05 12:34:56.1377,1903-12-25 +1903-05-05 12:34:56.1378,1903-12-25 +1903-05-05 12:34:56.1379,1903-12-25 +1903-05-05 12:34:56.138,1903-12-25 +1903-05-05 12:34:56.1381,1903-12-25 +1903-05-05 12:34:56.1382,1903-12-25 +1903-05-05 12:34:56.1383,1903-12-25 +1903-05-05 12:34:56.1384,1903-12-25 +1903-05-05 12:34:56.1385,1903-12-25 +1903-05-05 12:34:56.1386,1903-12-25 +1903-05-05 12:34:56.1387,1903-12-25 +1903-05-05 12:34:56.1388,1903-12-25 +1903-05-05 12:34:56.1389,1903-12-25 +1903-05-05 12:34:56.139,1903-12-25 +1903-05-05 12:34:56.1391,1903-12-25 +1903-05-05 12:34:56.1392,1903-12-25 +1903-05-05 12:34:56.1393,1903-12-25 +1903-05-05 12:34:56.1394,1903-12-25 +1903-05-05 12:34:56.1395,1903-12-25 +1903-05-05 12:34:56.1396,1903-12-25 +1903-05-05 12:34:56.1397,1903-12-25 +1903-05-05 12:34:56.1398,1903-12-25 +1903-05-05 12:34:56.1399,1903-12-25 +1903-05-05 12:34:56.14,1903-12-25 +1903-05-05 12:34:56.1401,1903-12-25 +1903-05-05 12:34:56.1402,1903-12-25 +1903-05-05 12:34:56.1403,1903-12-25 +1903-05-05 12:34:56.1404,1903-12-25 +1903-05-05 12:34:56.1405,1903-12-25 +1903-05-05 12:34:56.1406,1903-12-25 +1903-05-05 12:34:56.1407,1903-12-25 +1903-05-05 12:34:56.1408,1903-12-25 +1903-05-05 12:34:56.1409,1903-12-25 +1903-05-05 12:34:56.141,1903-12-25 +1903-05-05 12:34:56.1411,1903-12-25 +1903-05-05 12:34:56.1412,1903-12-25 +1903-05-05 12:34:56.1413,1903-12-25 +1903-05-05 12:34:56.1414,1903-12-25 +1903-05-05 12:34:56.1415,1903-12-25 +1903-05-05 12:34:56.1416,1903-12-25 +1903-05-05 12:34:56.1417,1903-12-25 +1903-05-05 12:34:56.1418,1903-12-25 +1903-05-05 12:34:56.1419,1903-12-25 +1903-05-05 12:34:56.142,1903-12-25 +1903-05-05 12:34:56.1421,1903-12-25 +1903-05-05 12:34:56.1422,1903-12-25 +1903-05-05 12:34:56.1423,1903-12-25 +1903-05-05 12:34:56.1424,1903-12-25 +1903-05-05 12:34:56.1425,1903-12-25 +1903-05-05 12:34:56.1426,1903-12-25 +1903-05-05 12:34:56.1427,1903-12-25 +1903-05-05 12:34:56.1428,1903-12-25 +1903-05-05 12:34:56.1429,1903-12-25 +1903-05-05 12:34:56.143,1903-12-25 +1903-05-05 12:34:56.1431,1903-12-25 +1903-05-05 12:34:56.1432,1903-12-25 +1903-05-05 12:34:56.1433,1903-12-25 +1903-05-05 12:34:56.1434,1903-12-25 +1903-05-05 12:34:56.1435,1903-12-25 +1903-05-05 12:34:56.1436,1903-12-25 +1903-05-05 12:34:56.1437,1903-12-25 +1903-05-05 12:34:56.1438,1903-12-25 +1903-05-05 12:34:56.1439,1903-12-25 +1903-05-05 12:34:56.144,1903-12-25 +1903-05-05 12:34:56.1441,1903-12-25 +1903-05-05 12:34:56.1442,1903-12-25 +1903-05-05 12:34:56.1443,1903-12-25 +1903-05-05 12:34:56.1444,1903-12-25 +1903-05-05 12:34:56.1445,1903-12-25 +1903-05-05 12:34:56.1446,1903-12-25 +1903-05-05 12:34:56.1447,1903-12-25 +1903-05-05 12:34:56.1448,1903-12-25 +1903-05-05 12:34:56.1449,1903-12-25 +1903-05-05 12:34:56.145,1903-12-25 +1903-05-05 12:34:56.1451,1903-12-25 +1903-05-05 12:34:56.1452,1903-12-25 +1903-05-05 12:34:56.1453,1903-12-25 +1903-05-05 12:34:56.1454,1903-12-25 +1903-05-05 12:34:56.1455,1903-12-25 +1903-05-05 12:34:56.1456,1903-12-25 +1903-05-05 12:34:56.1457,1903-12-25 +1903-05-05 12:34:56.1458,1903-12-25 +1903-05-05 12:34:56.1459,1903-12-25 +1903-05-05 12:34:56.146,1903-12-25 +1903-05-05 12:34:56.1461,1903-12-25 +1903-05-05 12:34:56.1462,1903-12-25 +1903-05-05 12:34:56.1463,1903-12-25 +1903-05-05 12:34:56.1464,1903-12-25 +1903-05-05 12:34:56.1465,1903-12-25 +1903-05-05 12:34:56.1466,1903-12-25 +1903-05-05 12:34:56.1467,1903-12-25 +1903-05-05 12:34:56.1468,1903-12-25 +1903-05-05 12:34:56.1469,1903-12-25 +1903-05-05 12:34:56.147,1903-12-25 +1903-05-05 12:34:56.1471,1903-12-25 +1903-05-05 12:34:56.1472,1903-12-25 +1903-05-05 12:34:56.1473,1903-12-25 +1903-05-05 12:34:56.1474,1903-12-25 +1903-05-05 12:34:56.1475,1903-12-25 +1903-05-05 12:34:56.1476,1903-12-25 +1903-05-05 12:34:56.1477,1903-12-25 +1903-05-05 12:34:56.1478,1903-12-25 +1903-05-05 12:34:56.1479,1903-12-25 +1903-05-05 12:34:56.148,1903-12-25 +1903-05-05 12:34:56.1481,1903-12-25 +1903-05-05 12:34:56.1482,1903-12-25 +1903-05-05 12:34:56.1483,1903-12-25 +1903-05-05 12:34:56.1484,1903-12-25 +1903-05-05 12:34:56.1485,1903-12-25 +1903-05-05 12:34:56.1486,1903-12-25 +1903-05-05 12:34:56.1487,1903-12-25 +1903-05-05 12:34:56.1488,1903-12-25 +1903-05-05 12:34:56.1489,1903-12-25 +1903-05-05 12:34:56.149,1903-12-25 +1903-05-05 12:34:56.1491,1903-12-25 +1903-05-05 12:34:56.1492,1903-12-25 +1903-05-05 12:34:56.1493,1903-12-25 +1903-05-05 12:34:56.1494,1903-12-25 +1903-05-05 12:34:56.1495,1903-12-25 +1903-05-05 12:34:56.1496,1903-12-25 +1903-05-05 12:34:56.1497,1903-12-25 +1903-05-05 12:34:56.1498,1903-12-25 +1903-05-05 12:34:56.1499,1903-12-25 +1903-05-05 12:34:56.15,1903-12-25 +1903-05-05 12:34:56.1501,1903-12-25 +1903-05-05 12:34:56.1502,1903-12-25 +1903-05-05 12:34:56.1503,1903-12-25 +1903-05-05 12:34:56.1504,1903-12-25 +1903-05-05 12:34:56.1505,1903-12-25 +1903-05-05 12:34:56.1506,1903-12-25 +1903-05-05 12:34:56.1507,1903-12-25 +1903-05-05 12:34:56.1508,1903-12-25 +1903-05-05 12:34:56.1509,1903-12-25 +1903-05-05 12:34:56.151,1903-12-25 +1903-05-05 12:34:56.1511,1903-12-25 +1903-05-05 12:34:56.1512,1903-12-25 +1903-05-05 12:34:56.1513,1903-12-25 +1903-05-05 12:34:56.1514,1903-12-25 +1903-05-05 12:34:56.1515,1903-12-25 +1903-05-05 12:34:56.1516,1903-12-25 +1903-05-05 12:34:56.1517,1903-12-25 +1903-05-05 12:34:56.1518,1903-12-25 +1903-05-05 12:34:56.1519,1903-12-25 +1903-05-05 12:34:56.152,1903-12-25 +1903-05-05 12:34:56.1521,1903-12-25 +1903-05-05 12:34:56.1522,1903-12-25 +1903-05-05 12:34:56.1523,1903-12-25 +1903-05-05 12:34:56.1524,1903-12-25 +1903-05-05 12:34:56.1525,1903-12-25 +1903-05-05 12:34:56.1526,1903-12-25 +1903-05-05 12:34:56.1527,1903-12-25 +1903-05-05 12:34:56.1528,1903-12-25 +1903-05-05 12:34:56.1529,1903-12-25 +1903-05-05 12:34:56.153,1903-12-25 +1903-05-05 12:34:56.1531,1903-12-25 +1903-05-05 12:34:56.1532,1903-12-25 +1903-05-05 12:34:56.1533,1903-12-25 +1903-05-05 12:34:56.1534,1903-12-25 +1903-05-05 12:34:56.1535,1903-12-25 +1903-05-05 12:34:56.1536,1903-12-25 +1903-05-05 12:34:56.1537,1903-12-25 +1903-05-05 12:34:56.1538,1903-12-25 +1903-05-05 12:34:56.1539,1903-12-25 +1903-05-05 12:34:56.154,1903-12-25 +1903-05-05 12:34:56.1541,1903-12-25 +1903-05-05 12:34:56.1542,1903-12-25 +1903-05-05 12:34:56.1543,1903-12-25 +1903-05-05 12:34:56.1544,1903-12-25 +1903-05-05 12:34:56.1545,1903-12-25 +1903-05-05 12:34:56.1546,1903-12-25 +1903-05-05 12:34:56.1547,1903-12-25 +1903-05-05 12:34:56.1548,1903-12-25 +1903-05-05 12:34:56.1549,1903-12-25 +1903-05-05 12:34:56.155,1903-12-25 +1903-05-05 12:34:56.1551,1903-12-25 +1903-05-05 12:34:56.1552,1903-12-25 +1903-05-05 12:34:56.1553,1903-12-25 +1903-05-05 12:34:56.1554,1903-12-25 +1903-05-05 12:34:56.1555,1903-12-25 +1903-05-05 12:34:56.1556,1903-12-25 +1903-05-05 12:34:56.1557,1903-12-25 +1903-05-05 12:34:56.1558,1903-12-25 +1903-05-05 12:34:56.1559,1903-12-25 +1903-05-05 12:34:56.156,1903-12-25 +1903-05-05 12:34:56.1561,1903-12-25 +1903-05-05 12:34:56.1562,1903-12-25 +1903-05-05 12:34:56.1563,1903-12-25 +1903-05-05 12:34:56.1564,1903-12-25 +1903-05-05 12:34:56.1565,1903-12-25 +1903-05-05 12:34:56.1566,1903-12-25 +1903-05-05 12:34:56.1567,1903-12-25 +1903-05-05 12:34:56.1568,1903-12-25 +1903-05-05 12:34:56.1569,1903-12-25 +1903-05-05 12:34:56.157,1903-12-25 +1903-05-05 12:34:56.1571,1903-12-25 +1903-05-05 12:34:56.1572,1903-12-25 +1903-05-05 12:34:56.1573,1903-12-25 +1903-05-05 12:34:56.1574,1903-12-25 +1903-05-05 12:34:56.1575,1903-12-25 +1903-05-05 12:34:56.1576,1903-12-25 +1903-05-05 12:34:56.1577,1903-12-25 +1903-05-05 12:34:56.1578,1903-12-25 +1903-05-05 12:34:56.1579,1903-12-25 +1903-05-05 12:34:56.158,1903-12-25 +1903-05-05 12:34:56.1581,1903-12-25 +1903-05-05 12:34:56.1582,1903-12-25 +1903-05-05 12:34:56.1583,1903-12-25 +1903-05-05 12:34:56.1584,1903-12-25 +1903-05-05 12:34:56.1585,1903-12-25 +1903-05-05 12:34:56.1586,1903-12-25 +1903-05-05 12:34:56.1587,1903-12-25 +1903-05-05 12:34:56.1588,1903-12-25 +1903-05-05 12:34:56.1589,1903-12-25 +1903-05-05 12:34:56.159,1903-12-25 +1903-05-05 12:34:56.1591,1903-12-25 +1903-05-05 12:34:56.1592,1903-12-25 +1903-05-05 12:34:56.1593,1903-12-25 +1903-05-05 12:34:56.1594,1903-12-25 +1903-05-05 12:34:56.1595,1903-12-25 +1903-05-05 12:34:56.1596,1903-12-25 +1903-05-05 12:34:56.1597,1903-12-25 +1903-05-05 12:34:56.1598,1903-12-25 +1903-05-05 12:34:56.1599,1903-12-25 +1903-05-05 12:34:56.16,1903-12-25 +1903-05-05 12:34:56.1601,1903-12-25 +1903-05-05 12:34:56.1602,1903-12-25 +1903-05-05 12:34:56.1603,1903-12-25 +1903-05-05 12:34:56.1604,1903-12-25 +1903-05-05 12:34:56.1605,1903-12-25 +1903-05-05 12:34:56.1606,1903-12-25 +1903-05-05 12:34:56.1607,1903-12-25 +1903-05-05 12:34:56.1608,1903-12-25 +1903-05-05 12:34:56.1609,1903-12-25 +1903-05-05 12:34:56.161,1903-12-25 +1903-05-05 12:34:56.1611,1903-12-25 +1903-05-05 12:34:56.1612,1903-12-25 +1903-05-05 12:34:56.1613,1903-12-25 +1903-05-05 12:34:56.1614,1903-12-25 +1903-05-05 12:34:56.1615,1903-12-25 +1903-05-05 12:34:56.1616,1903-12-25 +1903-05-05 12:34:56.1617,1903-12-25 +1903-05-05 12:34:56.1618,1903-12-25 +1903-05-05 12:34:56.1619,1903-12-25 +1903-05-05 12:34:56.162,1903-12-25 +1903-05-05 12:34:56.1621,1903-12-25 +1903-05-05 12:34:56.1622,1903-12-25 +1903-05-05 12:34:56.1623,1903-12-25 +1903-05-05 12:34:56.1624,1903-12-25 +1903-05-05 12:34:56.1625,1903-12-25 +1903-05-05 12:34:56.1626,1903-12-25 +1903-05-05 12:34:56.1627,1903-12-25 +1903-05-05 12:34:56.1628,1903-12-25 +1903-05-05 12:34:56.1629,1903-12-25 +1903-05-05 12:34:56.163,1903-12-25 +1903-05-05 12:34:56.1631,1903-12-25 +1903-05-05 12:34:56.1632,1903-12-25 +1903-05-05 12:34:56.1633,1903-12-25 +1903-05-05 12:34:56.1634,1903-12-25 +1903-05-05 12:34:56.1635,1903-12-25 +1903-05-05 12:34:56.1636,1903-12-25 +1903-05-05 12:34:56.1637,1903-12-25 +1903-05-05 12:34:56.1638,1903-12-25 +1903-05-05 12:34:56.1639,1903-12-25 +1903-05-05 12:34:56.164,1903-12-25 +1903-05-05 12:34:56.1641,1903-12-25 +1903-05-05 12:34:56.1642,1903-12-25 +1903-05-05 12:34:56.1643,1903-12-25 +1903-05-05 12:34:56.1644,1903-12-25 +1903-05-05 12:34:56.1645,1903-12-25 +1903-05-05 12:34:56.1646,1903-12-25 +1903-05-05 12:34:56.1647,1903-12-25 +1903-05-05 12:34:56.1648,1903-12-25 +1903-05-05 12:34:56.1649,1903-12-25 +1903-05-05 12:34:56.165,1903-12-25 +1903-05-05 12:34:56.1651,1903-12-25 +1903-05-05 12:34:56.1652,1903-12-25 +1903-05-05 12:34:56.1653,1903-12-25 +1903-05-05 12:34:56.1654,1903-12-25 +1903-05-05 12:34:56.1655,1903-12-25 +1903-05-05 12:34:56.1656,1903-12-25 +1903-05-05 12:34:56.1657,1903-12-25 +1903-05-05 12:34:56.1658,1903-12-25 +1903-05-05 12:34:56.1659,1903-12-25 +1903-05-05 12:34:56.166,1903-12-25 +1903-05-05 12:34:56.1661,1903-12-25 +1903-05-05 12:34:56.1662,1903-12-25 +1903-05-05 12:34:56.1663,1903-12-25 +1903-05-05 12:34:56.1664,1903-12-25 +1903-05-05 12:34:56.1665,1903-12-25 +1903-05-05 12:34:56.1666,1903-12-25 +1903-05-05 12:34:56.1667,1903-12-25 +1903-05-05 12:34:56.1668,1903-12-25 +1903-05-05 12:34:56.1669,1903-12-25 +1903-05-05 12:34:56.167,1903-12-25 +1903-05-05 12:34:56.1671,1903-12-25 +1903-05-05 12:34:56.1672,1903-12-25 +1903-05-05 12:34:56.1673,1903-12-25 +1903-05-05 12:34:56.1674,1903-12-25 +1903-05-05 12:34:56.1675,1903-12-25 +1903-05-05 12:34:56.1676,1903-12-25 +1903-05-05 12:34:56.1677,1903-12-25 +1903-05-05 12:34:56.1678,1903-12-25 +1903-05-05 12:34:56.1679,1903-12-25 +1903-05-05 12:34:56.168,1903-12-25 +1903-05-05 12:34:56.1681,1903-12-25 +1903-05-05 12:34:56.1682,1903-12-25 +1903-05-05 12:34:56.1683,1903-12-25 +1903-05-05 12:34:56.1684,1903-12-25 +1903-05-05 12:34:56.1685,1903-12-25 +1903-05-05 12:34:56.1686,1903-12-25 +1903-05-05 12:34:56.1687,1903-12-25 +1903-05-05 12:34:56.1688,1903-12-25 +1903-05-05 12:34:56.1689,1903-12-25 +1903-05-05 12:34:56.169,1903-12-25 +1903-05-05 12:34:56.1691,1903-12-25 +1903-05-05 12:34:56.1692,1903-12-25 +1903-05-05 12:34:56.1693,1903-12-25 +1903-05-05 12:34:56.1694,1903-12-25 +1903-05-05 12:34:56.1695,1903-12-25 +1903-05-05 12:34:56.1696,1903-12-25 +1903-05-05 12:34:56.1697,1903-12-25 +1903-05-05 12:34:56.1698,1903-12-25 +1903-05-05 12:34:56.1699,1903-12-25 +1903-05-05 12:34:56.17,1903-12-25 +1903-05-05 12:34:56.1701,1903-12-25 +1903-05-05 12:34:56.1702,1903-12-25 +1903-05-05 12:34:56.1703,1903-12-25 +1903-05-05 12:34:56.1704,1903-12-25 +1903-05-05 12:34:56.1705,1903-12-25 +1903-05-05 12:34:56.1706,1903-12-25 +1903-05-05 12:34:56.1707,1903-12-25 +1903-05-05 12:34:56.1708,1903-12-25 +1903-05-05 12:34:56.1709,1903-12-25 +1903-05-05 12:34:56.171,1903-12-25 +1903-05-05 12:34:56.1711,1903-12-25 +1903-05-05 12:34:56.1712,1903-12-25 +1903-05-05 12:34:56.1713,1903-12-25 +1903-05-05 12:34:56.1714,1903-12-25 +1903-05-05 12:34:56.1715,1903-12-25 +1903-05-05 12:34:56.1716,1903-12-25 +1903-05-05 12:34:56.1717,1903-12-25 +1903-05-05 12:34:56.1718,1903-12-25 +1903-05-05 12:34:56.1719,1903-12-25 +1903-05-05 12:34:56.172,1903-12-25 +1903-05-05 12:34:56.1721,1903-12-25 +1903-05-05 12:34:56.1722,1903-12-25 +1903-05-05 12:34:56.1723,1903-12-25 +1903-05-05 12:34:56.1724,1903-12-25 +1903-05-05 12:34:56.1725,1903-12-25 +1903-05-05 12:34:56.1726,1903-12-25 +1903-05-05 12:34:56.1727,1903-12-25 +1903-05-05 12:34:56.1728,1903-12-25 +1903-05-05 12:34:56.1729,1903-12-25 +1903-05-05 12:34:56.173,1903-12-25 +1903-05-05 12:34:56.1731,1903-12-25 +1903-05-05 12:34:56.1732,1903-12-25 +1903-05-05 12:34:56.1733,1903-12-25 +1903-05-05 12:34:56.1734,1903-12-25 +1903-05-05 12:34:56.1735,1903-12-25 +1903-05-05 12:34:56.1736,1903-12-25 +1903-05-05 12:34:56.1737,1903-12-25 +1903-05-05 12:34:56.1738,1903-12-25 +1903-05-05 12:34:56.1739,1903-12-25 +1903-05-05 12:34:56.174,1903-12-25 +1903-05-05 12:34:56.1741,1903-12-25 +1903-05-05 12:34:56.1742,1903-12-25 +1903-05-05 12:34:56.1743,1903-12-25 +1903-05-05 12:34:56.1744,1903-12-25 +1903-05-05 12:34:56.1745,1903-12-25 +1903-05-05 12:34:56.1746,1903-12-25 +1903-05-05 12:34:56.1747,1903-12-25 +1903-05-05 12:34:56.1748,1903-12-25 +1903-05-05 12:34:56.1749,1903-12-25 +1903-05-05 12:34:56.175,1903-12-25 +1903-05-05 12:34:56.1751,1903-12-25 +1903-05-05 12:34:56.1752,1903-12-25 +1903-05-05 12:34:56.1753,1903-12-25 +1903-05-05 12:34:56.1754,1903-12-25 +1903-05-05 12:34:56.1755,1903-12-25 +1903-05-05 12:34:56.1756,1903-12-25 +1903-05-05 12:34:56.1757,1903-12-25 +1903-05-05 12:34:56.1758,1903-12-25 +1903-05-05 12:34:56.1759,1903-12-25 +1903-05-05 12:34:56.176,1903-12-25 +1903-05-05 12:34:56.1761,1903-12-25 +1903-05-05 12:34:56.1762,1903-12-25 +1903-05-05 12:34:56.1763,1903-12-25 +1903-05-05 12:34:56.1764,1903-12-25 +1903-05-05 12:34:56.1765,1903-12-25 +1903-05-05 12:34:56.1766,1903-12-25 +1903-05-05 12:34:56.1767,1903-12-25 +1903-05-05 12:34:56.1768,1903-12-25 +1903-05-05 12:34:56.1769,1903-12-25 +1903-05-05 12:34:56.177,1903-12-25 +1903-05-05 12:34:56.1771,1903-12-25 +1903-05-05 12:34:56.1772,1903-12-25 +1903-05-05 12:34:56.1773,1903-12-25 +1903-05-05 12:34:56.1774,1903-12-25 +1903-05-05 12:34:56.1775,1903-12-25 +1903-05-05 12:34:56.1776,1903-12-25 +1903-05-05 12:34:56.1777,1903-12-25 +1903-05-05 12:34:56.1778,1903-12-25 +1903-05-05 12:34:56.1779,1903-12-25 +1903-05-05 12:34:56.178,1903-12-25 +1903-05-05 12:34:56.1781,1903-12-25 +1903-05-05 12:34:56.1782,1903-12-25 +1903-05-05 12:34:56.1783,1903-12-25 +1903-05-05 12:34:56.1784,1903-12-25 +1903-05-05 12:34:56.1785,1903-12-25 +1903-05-05 12:34:56.1786,1903-12-25 +1903-05-05 12:34:56.1787,1903-12-25 +1903-05-05 12:34:56.1788,1903-12-25 +1903-05-05 12:34:56.1789,1903-12-25 +1903-05-05 12:34:56.179,1903-12-25 +1903-05-05 12:34:56.1791,1903-12-25 +1903-05-05 12:34:56.1792,1903-12-25 +1903-05-05 12:34:56.1793,1903-12-25 +1903-05-05 12:34:56.1794,1903-12-25 +1903-05-05 12:34:56.1795,1903-12-25 +1903-05-05 12:34:56.1796,1903-12-25 +1903-05-05 12:34:56.1797,1903-12-25 +1903-05-05 12:34:56.1798,1903-12-25 +1903-05-05 12:34:56.1799,1903-12-25 +1903-05-05 12:34:56.18,1903-12-25 +1903-05-05 12:34:56.1801,1903-12-25 +1903-05-05 12:34:56.1802,1903-12-25 +1903-05-05 12:34:56.1803,1903-12-25 +1903-05-05 12:34:56.1804,1903-12-25 +1903-05-05 12:34:56.1805,1903-12-25 +1903-05-05 12:34:56.1806,1903-12-25 +1903-05-05 12:34:56.1807,1903-12-25 +1903-05-05 12:34:56.1808,1903-12-25 +1903-05-05 12:34:56.1809,1903-12-25 +1903-05-05 12:34:56.181,1903-12-25 +1903-05-05 12:34:56.1811,1903-12-25 +1903-05-05 12:34:56.1812,1903-12-25 +1903-05-05 12:34:56.1813,1903-12-25 +1903-05-05 12:34:56.1814,1903-12-25 +1903-05-05 12:34:56.1815,1903-12-25 +1903-05-05 12:34:56.1816,1903-12-25 +1903-05-05 12:34:56.1817,1903-12-25 +1903-05-05 12:34:56.1818,1903-12-25 +1903-05-05 12:34:56.1819,1903-12-25 +1903-05-05 12:34:56.182,1903-12-25 +1903-05-05 12:34:56.1821,1903-12-25 +1903-05-05 12:34:56.1822,1903-12-25 +1903-05-05 12:34:56.1823,1903-12-25 +1903-05-05 12:34:56.1824,1903-12-25 +1903-05-05 12:34:56.1825,1903-12-25 +1903-05-05 12:34:56.1826,1903-12-25 +1903-05-05 12:34:56.1827,1903-12-25 +1903-05-05 12:34:56.1828,1903-12-25 +1903-05-05 12:34:56.1829,1903-12-25 +1903-05-05 12:34:56.183,1903-12-25 +1903-05-05 12:34:56.1831,1903-12-25 +1903-05-05 12:34:56.1832,1903-12-25 +1903-05-05 12:34:56.1833,1903-12-25 +1903-05-05 12:34:56.1834,1903-12-25 +1903-05-05 12:34:56.1835,1903-12-25 +1903-05-05 12:34:56.1836,1903-12-25 +1903-05-05 12:34:56.1837,1903-12-25 +1903-05-05 12:34:56.1838,1903-12-25 +1903-05-05 12:34:56.1839,1903-12-25 +1903-05-05 12:34:56.184,1903-12-25 +1903-05-05 12:34:56.1841,1903-12-25 +1903-05-05 12:34:56.1842,1903-12-25 +1903-05-05 12:34:56.1843,1903-12-25 +1903-05-05 12:34:56.1844,1903-12-25 +1903-05-05 12:34:56.1845,1903-12-25 +1903-05-05 12:34:56.1846,1903-12-25 +1903-05-05 12:34:56.1847,1903-12-25 +1903-05-05 12:34:56.1848,1903-12-25 +1903-05-05 12:34:56.1849,1903-12-25 +1903-05-05 12:34:56.185,1903-12-25 +1903-05-05 12:34:56.1851,1903-12-25 +1903-05-05 12:34:56.1852,1903-12-25 +1903-05-05 12:34:56.1853,1903-12-25 +1903-05-05 12:34:56.1854,1903-12-25 +1903-05-05 12:34:56.1855,1903-12-25 +1903-05-05 12:34:56.1856,1903-12-25 +1903-05-05 12:34:56.1857,1903-12-25 +1903-05-05 12:34:56.1858,1903-12-25 +1903-05-05 12:34:56.1859,1903-12-25 +1903-05-05 12:34:56.186,1903-12-25 +1903-05-05 12:34:56.1861,1903-12-25 +1903-05-05 12:34:56.1862,1903-12-25 +1903-05-05 12:34:56.1863,1903-12-25 +1903-05-05 12:34:56.1864,1903-12-25 +1903-05-05 12:34:56.1865,1903-12-25 +1903-05-05 12:34:56.1866,1903-12-25 +1903-05-05 12:34:56.1867,1903-12-25 +1903-05-05 12:34:56.1868,1903-12-25 +1903-05-05 12:34:56.1869,1903-12-25 +1903-05-05 12:34:56.187,1903-12-25 +1903-05-05 12:34:56.1871,1903-12-25 +1903-05-05 12:34:56.1872,1903-12-25 +1903-05-05 12:34:56.1873,1903-12-25 +1903-05-05 12:34:56.1874,1903-12-25 +1903-05-05 12:34:56.1875,1903-12-25 +1903-05-05 12:34:56.1876,1903-12-25 +1903-05-05 12:34:56.1877,1903-12-25 +1903-05-05 12:34:56.1878,1903-12-25 +1903-05-05 12:34:56.1879,1903-12-25 +1903-05-05 12:34:56.188,1903-12-25 +1903-05-05 12:34:56.1881,1903-12-25 +1903-05-05 12:34:56.1882,1903-12-25 +1903-05-05 12:34:56.1883,1903-12-25 +1903-05-05 12:34:56.1884,1903-12-25 +1903-05-05 12:34:56.1885,1903-12-25 +1903-05-05 12:34:56.1886,1903-12-25 +1903-05-05 12:34:56.1887,1903-12-25 +1903-05-05 12:34:56.1888,1903-12-25 +1903-05-05 12:34:56.1889,1903-12-25 +1903-05-05 12:34:56.189,1903-12-25 +1903-05-05 12:34:56.1891,1903-12-25 +1903-05-05 12:34:56.1892,1903-12-25 +1903-05-05 12:34:56.1893,1903-12-25 +1903-05-05 12:34:56.1894,1903-12-25 +1903-05-05 12:34:56.1895,1903-12-25 +1903-05-05 12:34:56.1896,1903-12-25 +1903-05-05 12:34:56.1897,1903-12-25 +1903-05-05 12:34:56.1898,1903-12-25 +1903-05-05 12:34:56.1899,1903-12-25 +1903-05-05 12:34:56.19,1903-12-25 +1903-05-05 12:34:56.1901,1903-12-25 +1903-05-05 12:34:56.1902,1903-12-25 +1903-05-05 12:34:56.1903,1903-12-25 +1903-05-05 12:34:56.1904,1903-12-25 +1903-05-05 12:34:56.1905,1903-12-25 +1903-05-05 12:34:56.1906,1903-12-25 +1903-05-05 12:34:56.1907,1903-12-25 +1903-05-05 12:34:56.1908,1903-12-25 +1903-05-05 12:34:56.1909,1903-12-25 +1903-05-05 12:34:56.191,1903-12-25 +1903-05-05 12:34:56.1911,1903-12-25 +1903-05-05 12:34:56.1912,1903-12-25 +1903-05-05 12:34:56.1913,1903-12-25 +1903-05-05 12:34:56.1914,1903-12-25 +1903-05-05 12:34:56.1915,1903-12-25 +1903-05-05 12:34:56.1916,1903-12-25 +1903-05-05 12:34:56.1917,1903-12-25 +1903-05-05 12:34:56.1918,1903-12-25 +1903-05-05 12:34:56.1919,1903-12-25 +1903-05-05 12:34:56.192,1903-12-25 +1903-05-05 12:34:56.1921,1903-12-25 +1903-05-05 12:34:56.1922,1903-12-25 +1903-05-05 12:34:56.1923,1903-12-25 +1903-05-05 12:34:56.1924,1903-12-25 +1903-05-05 12:34:56.1925,1903-12-25 +1903-05-05 12:34:56.1926,1903-12-25 +1903-05-05 12:34:56.1927,1903-12-25 +1903-05-05 12:34:56.1928,1903-12-25 +1903-05-05 12:34:56.1929,1903-12-25 +1903-05-05 12:34:56.193,1903-12-25 +1903-05-05 12:34:56.1931,1903-12-25 +1903-05-05 12:34:56.1932,1903-12-25 +1903-05-05 12:34:56.1933,1903-12-25 +1903-05-05 12:34:56.1934,1903-12-25 +1903-05-05 12:34:56.1935,1903-12-25 +1903-05-05 12:34:56.1936,1903-12-25 +1903-05-05 12:34:56.1937,1903-12-25 +1903-05-05 12:34:56.1938,1903-12-25 +1903-05-05 12:34:56.1939,1903-12-25 +1903-05-05 12:34:56.194,1903-12-25 +1903-05-05 12:34:56.1941,1903-12-25 +1903-05-05 12:34:56.1942,1903-12-25 +1903-05-05 12:34:56.1943,1903-12-25 +1903-05-05 12:34:56.1944,1903-12-25 +1903-05-05 12:34:56.1945,1903-12-25 +1903-05-05 12:34:56.1946,1903-12-25 +1903-05-05 12:34:56.1947,1903-12-25 +1903-05-05 12:34:56.1948,1903-12-25 +1903-05-05 12:34:56.1949,1903-12-25 +1903-05-05 12:34:56.195,1903-12-25 +1903-05-05 12:34:56.1951,1903-12-25 +1903-05-05 12:34:56.1952,1903-12-25 +1903-05-05 12:34:56.1953,1903-12-25 +1903-05-05 12:34:56.1954,1903-12-25 +1903-05-05 12:34:56.1955,1903-12-25 +1903-05-05 12:34:56.1956,1903-12-25 +1903-05-05 12:34:56.1957,1903-12-25 +1903-05-05 12:34:56.1958,1903-12-25 +1903-05-05 12:34:56.1959,1903-12-25 +1903-05-05 12:34:56.196,1903-12-25 +1903-05-05 12:34:56.1961,1903-12-25 +1903-05-05 12:34:56.1962,1903-12-25 +1903-05-05 12:34:56.1963,1903-12-25 +1903-05-05 12:34:56.1964,1903-12-25 +1903-05-05 12:34:56.1965,1903-12-25 +1903-05-05 12:34:56.1966,1903-12-25 +1903-05-05 12:34:56.1967,1903-12-25 +1903-05-05 12:34:56.1968,1903-12-25 +1903-05-05 12:34:56.1969,1903-12-25 +1903-05-05 12:34:56.197,1903-12-25 +1903-05-05 12:34:56.1971,1903-12-25 +1903-05-05 12:34:56.1972,1903-12-25 +1903-05-05 12:34:56.1973,1903-12-25 +1903-05-05 12:34:56.1974,1903-12-25 +1903-05-05 12:34:56.1975,1903-12-25 +1903-05-05 12:34:56.1976,1903-12-25 +1903-05-05 12:34:56.1977,1903-12-25 +1903-05-05 12:34:56.1978,1903-12-25 +1903-05-05 12:34:56.1979,1903-12-25 +1903-05-05 12:34:56.198,1903-12-25 +1903-05-05 12:34:56.1981,1903-12-25 +1903-05-05 12:34:56.1982,1903-12-25 +1903-05-05 12:34:56.1983,1903-12-25 +1903-05-05 12:34:56.1984,1903-12-25 +1903-05-05 12:34:56.1985,1903-12-25 +1903-05-05 12:34:56.1986,1903-12-25 +1903-05-05 12:34:56.1987,1903-12-25 +1903-05-05 12:34:56.1988,1903-12-25 +1903-05-05 12:34:56.1989,1903-12-25 +1903-05-05 12:34:56.199,1903-12-25 +1903-05-05 12:34:56.1991,1903-12-25 +1903-05-05 12:34:56.1992,1903-12-25 +1903-05-05 12:34:56.1993,1903-12-25 +1903-05-05 12:34:56.1994,1903-12-25 +1903-05-05 12:34:56.1995,1903-12-25 +1903-05-05 12:34:56.1996,1903-12-25 +1903-05-05 12:34:56.1997,1903-12-25 +1903-05-05 12:34:56.1998,1903-12-25 +1903-05-05 12:34:56.1999,1903-12-25 +1904-05-05 12:34:56.1,1904-12-25 +1904-05-05 12:34:56.1001,1904-12-25 +1904-05-05 12:34:56.1002,1904-12-25 +1904-05-05 12:34:56.1003,1904-12-25 +1904-05-05 12:34:56.1004,1904-12-25 +1904-05-05 12:34:56.1005,1904-12-25 +1904-05-05 12:34:56.1006,1904-12-25 +1904-05-05 12:34:56.1007,1904-12-25 +1904-05-05 12:34:56.1008,1904-12-25 +1904-05-05 12:34:56.1009,1904-12-25 +1904-05-05 12:34:56.101,1904-12-25 +1904-05-05 12:34:56.1011,1904-12-25 +1904-05-05 12:34:56.1012,1904-12-25 +1904-05-05 12:34:56.1013,1904-12-25 +1904-05-05 12:34:56.1014,1904-12-25 +1904-05-05 12:34:56.1015,1904-12-25 +1904-05-05 12:34:56.1016,1904-12-25 +1904-05-05 12:34:56.1017,1904-12-25 +1904-05-05 12:34:56.1018,1904-12-25 +1904-05-05 12:34:56.1019,1904-12-25 +1904-05-05 12:34:56.102,1904-12-25 +1904-05-05 12:34:56.1021,1904-12-25 +1904-05-05 12:34:56.1022,1904-12-25 +1904-05-05 12:34:56.1023,1904-12-25 +1904-05-05 12:34:56.1024,1904-12-25 +1904-05-05 12:34:56.1025,1904-12-25 +1904-05-05 12:34:56.1026,1904-12-25 +1904-05-05 12:34:56.1027,1904-12-25 +1904-05-05 12:34:56.1028,1904-12-25 +1904-05-05 12:34:56.1029,1904-12-25 +1904-05-05 12:34:56.103,1904-12-25 +1904-05-05 12:34:56.1031,1904-12-25 +1904-05-05 12:34:56.1032,1904-12-25 +1904-05-05 12:34:56.1033,1904-12-25 +1904-05-05 12:34:56.1034,1904-12-25 +1904-05-05 12:34:56.1035,1904-12-25 +1904-05-05 12:34:56.1036,1904-12-25 +1904-05-05 12:34:56.1037,1904-12-25 +1904-05-05 12:34:56.1038,1904-12-25 +1904-05-05 12:34:56.1039,1904-12-25 +1904-05-05 12:34:56.104,1904-12-25 +1904-05-05 12:34:56.1041,1904-12-25 +1904-05-05 12:34:56.1042,1904-12-25 +1904-05-05 12:34:56.1043,1904-12-25 +1904-05-05 12:34:56.1044,1904-12-25 +1904-05-05 12:34:56.1045,1904-12-25 +1904-05-05 12:34:56.1046,1904-12-25 +1904-05-05 12:34:56.1047,1904-12-25 +1904-05-05 12:34:56.1048,1904-12-25 +1904-05-05 12:34:56.1049,1904-12-25 +1904-05-05 12:34:56.105,1904-12-25 +1904-05-05 12:34:56.1051,1904-12-25 +1904-05-05 12:34:56.1052,1904-12-25 +1904-05-05 12:34:56.1053,1904-12-25 +1904-05-05 12:34:56.1054,1904-12-25 +1904-05-05 12:34:56.1055,1904-12-25 +1904-05-05 12:34:56.1056,1904-12-25 +1904-05-05 12:34:56.1057,1904-12-25 +1904-05-05 12:34:56.1058,1904-12-25 +1904-05-05 12:34:56.1059,1904-12-25 +1904-05-05 12:34:56.106,1904-12-25 +1904-05-05 12:34:56.1061,1904-12-25 +1904-05-05 12:34:56.1062,1904-12-25 +1904-05-05 12:34:56.1063,1904-12-25 +1904-05-05 12:34:56.1064,1904-12-25 +1904-05-05 12:34:56.1065,1904-12-25 +1904-05-05 12:34:56.1066,1904-12-25 +1904-05-05 12:34:56.1067,1904-12-25 +1904-05-05 12:34:56.1068,1904-12-25 +1904-05-05 12:34:56.1069,1904-12-25 +1904-05-05 12:34:56.107,1904-12-25 +1904-05-05 12:34:56.1071,1904-12-25 +1904-05-05 12:34:56.1072,1904-12-25 +1904-05-05 12:34:56.1073,1904-12-25 +1904-05-05 12:34:56.1074,1904-12-25 +1904-05-05 12:34:56.1075,1904-12-25 +1904-05-05 12:34:56.1076,1904-12-25 +1904-05-05 12:34:56.1077,1904-12-25 +1904-05-05 12:34:56.1078,1904-12-25 +1904-05-05 12:34:56.1079,1904-12-25 +1904-05-05 12:34:56.108,1904-12-25 +1904-05-05 12:34:56.1081,1904-12-25 +1904-05-05 12:34:56.1082,1904-12-25 +1904-05-05 12:34:56.1083,1904-12-25 +1904-05-05 12:34:56.1084,1904-12-25 +1904-05-05 12:34:56.1085,1904-12-25 +1904-05-05 12:34:56.1086,1904-12-25 +1904-05-05 12:34:56.1087,1904-12-25 +1904-05-05 12:34:56.1088,1904-12-25 +1904-05-05 12:34:56.1089,1904-12-25 +1904-05-05 12:34:56.109,1904-12-25 +1904-05-05 12:34:56.1091,1904-12-25 +1904-05-05 12:34:56.1092,1904-12-25 +1904-05-05 12:34:56.1093,1904-12-25 +1904-05-05 12:34:56.1094,1904-12-25 +1904-05-05 12:34:56.1095,1904-12-25 +1904-05-05 12:34:56.1096,1904-12-25 +1904-05-05 12:34:56.1097,1904-12-25 +1904-05-05 12:34:56.1098,1904-12-25 +1904-05-05 12:34:56.1099,1904-12-25 +1904-05-05 12:34:56.11,1904-12-25 +1904-05-05 12:34:56.1101,1904-12-25 +1904-05-05 12:34:56.1102,1904-12-25 +1904-05-05 12:34:56.1103,1904-12-25 +1904-05-05 12:34:56.1104,1904-12-25 +1904-05-05 12:34:56.1105,1904-12-25 +1904-05-05 12:34:56.1106,1904-12-25 +1904-05-05 12:34:56.1107,1904-12-25 +1904-05-05 12:34:56.1108,1904-12-25 +1904-05-05 12:34:56.1109,1904-12-25 +1904-05-05 12:34:56.111,1904-12-25 +1904-05-05 12:34:56.1111,1904-12-25 +1904-05-05 12:34:56.1112,1904-12-25 +1904-05-05 12:34:56.1113,1904-12-25 +1904-05-05 12:34:56.1114,1904-12-25 +1904-05-05 12:34:56.1115,1904-12-25 +1904-05-05 12:34:56.1116,1904-12-25 +1904-05-05 12:34:56.1117,1904-12-25 +1904-05-05 12:34:56.1118,1904-12-25 +1904-05-05 12:34:56.1119,1904-12-25 +1904-05-05 12:34:56.112,1904-12-25 +1904-05-05 12:34:56.1121,1904-12-25 +1904-05-05 12:34:56.1122,1904-12-25 +1904-05-05 12:34:56.1123,1904-12-25 +1904-05-05 12:34:56.1124,1904-12-25 +1904-05-05 12:34:56.1125,1904-12-25 +1904-05-05 12:34:56.1126,1904-12-25 +1904-05-05 12:34:56.1127,1904-12-25 +1904-05-05 12:34:56.1128,1904-12-25 +1904-05-05 12:34:56.1129,1904-12-25 +1904-05-05 12:34:56.113,1904-12-25 +1904-05-05 12:34:56.1131,1904-12-25 +1904-05-05 12:34:56.1132,1904-12-25 +1904-05-05 12:34:56.1133,1904-12-25 +1904-05-05 12:34:56.1134,1904-12-25 +1904-05-05 12:34:56.1135,1904-12-25 +1904-05-05 12:34:56.1136,1904-12-25 +1904-05-05 12:34:56.1137,1904-12-25 +1904-05-05 12:34:56.1138,1904-12-25 +1904-05-05 12:34:56.1139,1904-12-25 +1904-05-05 12:34:56.114,1904-12-25 +1904-05-05 12:34:56.1141,1904-12-25 +1904-05-05 12:34:56.1142,1904-12-25 +1904-05-05 12:34:56.1143,1904-12-25 +1904-05-05 12:34:56.1144,1904-12-25 +1904-05-05 12:34:56.1145,1904-12-25 +1904-05-05 12:34:56.1146,1904-12-25 +1904-05-05 12:34:56.1147,1904-12-25 +1904-05-05 12:34:56.1148,1904-12-25 +1904-05-05 12:34:56.1149,1904-12-25 +1904-05-05 12:34:56.115,1904-12-25 +1904-05-05 12:34:56.1151,1904-12-25 +1904-05-05 12:34:56.1152,1904-12-25 +1904-05-05 12:34:56.1153,1904-12-25 +1904-05-05 12:34:56.1154,1904-12-25 +1904-05-05 12:34:56.1155,1904-12-25 +1904-05-05 12:34:56.1156,1904-12-25 +1904-05-05 12:34:56.1157,1904-12-25 +1904-05-05 12:34:56.1158,1904-12-25 +1904-05-05 12:34:56.1159,1904-12-25 +1904-05-05 12:34:56.116,1904-12-25 +1904-05-05 12:34:56.1161,1904-12-25 +1904-05-05 12:34:56.1162,1904-12-25 +1904-05-05 12:34:56.1163,1904-12-25 +1904-05-05 12:34:56.1164,1904-12-25 +1904-05-05 12:34:56.1165,1904-12-25 +1904-05-05 12:34:56.1166,1904-12-25 +1904-05-05 12:34:56.1167,1904-12-25 +1904-05-05 12:34:56.1168,1904-12-25 +1904-05-05 12:34:56.1169,1904-12-25 +1904-05-05 12:34:56.117,1904-12-25 +1904-05-05 12:34:56.1171,1904-12-25 +1904-05-05 12:34:56.1172,1904-12-25 +1904-05-05 12:34:56.1173,1904-12-25 +1904-05-05 12:34:56.1174,1904-12-25 +1904-05-05 12:34:56.1175,1904-12-25 +1904-05-05 12:34:56.1176,1904-12-25 +1904-05-05 12:34:56.1177,1904-12-25 +1904-05-05 12:34:56.1178,1904-12-25 +1904-05-05 12:34:56.1179,1904-12-25 +1904-05-05 12:34:56.118,1904-12-25 +1904-05-05 12:34:56.1181,1904-12-25 +1904-05-05 12:34:56.1182,1904-12-25 +1904-05-05 12:34:56.1183,1904-12-25 +1904-05-05 12:34:56.1184,1904-12-25 +1904-05-05 12:34:56.1185,1904-12-25 +1904-05-05 12:34:56.1186,1904-12-25 +1904-05-05 12:34:56.1187,1904-12-25 +1904-05-05 12:34:56.1188,1904-12-25 +1904-05-05 12:34:56.1189,1904-12-25 +1904-05-05 12:34:56.119,1904-12-25 +1904-05-05 12:34:56.1191,1904-12-25 +1904-05-05 12:34:56.1192,1904-12-25 +1904-05-05 12:34:56.1193,1904-12-25 +1904-05-05 12:34:56.1194,1904-12-25 +1904-05-05 12:34:56.1195,1904-12-25 +1904-05-05 12:34:56.1196,1904-12-25 +1904-05-05 12:34:56.1197,1904-12-25 +1904-05-05 12:34:56.1198,1904-12-25 +1904-05-05 12:34:56.1199,1904-12-25 +1904-05-05 12:34:56.12,1904-12-25 +1904-05-05 12:34:56.1201,1904-12-25 +1904-05-05 12:34:56.1202,1904-12-25 +1904-05-05 12:34:56.1203,1904-12-25 +1904-05-05 12:34:56.1204,1904-12-25 +1904-05-05 12:34:56.1205,1904-12-25 +1904-05-05 12:34:56.1206,1904-12-25 +1904-05-05 12:34:56.1207,1904-12-25 +1904-05-05 12:34:56.1208,1904-12-25 +1904-05-05 12:34:56.1209,1904-12-25 +1904-05-05 12:34:56.121,1904-12-25 +1904-05-05 12:34:56.1211,1904-12-25 +1904-05-05 12:34:56.1212,1904-12-25 +1904-05-05 12:34:56.1213,1904-12-25 +1904-05-05 12:34:56.1214,1904-12-25 +1904-05-05 12:34:56.1215,1904-12-25 +1904-05-05 12:34:56.1216,1904-12-25 +1904-05-05 12:34:56.1217,1904-12-25 +1904-05-05 12:34:56.1218,1904-12-25 +1904-05-05 12:34:56.1219,1904-12-25 +1904-05-05 12:34:56.122,1904-12-25 +1904-05-05 12:34:56.1221,1904-12-25 +1904-05-05 12:34:56.1222,1904-12-25 +1904-05-05 12:34:56.1223,1904-12-25 +1904-05-05 12:34:56.1224,1904-12-25 +1904-05-05 12:34:56.1225,1904-12-25 +1904-05-05 12:34:56.1226,1904-12-25 +1904-05-05 12:34:56.1227,1904-12-25 +1904-05-05 12:34:56.1228,1904-12-25 +1904-05-05 12:34:56.1229,1904-12-25 +1904-05-05 12:34:56.123,1904-12-25 +1904-05-05 12:34:56.1231,1904-12-25 +1904-05-05 12:34:56.1232,1904-12-25 +1904-05-05 12:34:56.1233,1904-12-25 +1904-05-05 12:34:56.1234,1904-12-25 +1904-05-05 12:34:56.1235,1904-12-25 +1904-05-05 12:34:56.1236,1904-12-25 +1904-05-05 12:34:56.1237,1904-12-25 +1904-05-05 12:34:56.1238,1904-12-25 +1904-05-05 12:34:56.1239,1904-12-25 +1904-05-05 12:34:56.124,1904-12-25 +1904-05-05 12:34:56.1241,1904-12-25 +1904-05-05 12:34:56.1242,1904-12-25 +1904-05-05 12:34:56.1243,1904-12-25 +1904-05-05 12:34:56.1244,1904-12-25 +1904-05-05 12:34:56.1245,1904-12-25 +1904-05-05 12:34:56.1246,1904-12-25 +1904-05-05 12:34:56.1247,1904-12-25 +1904-05-05 12:34:56.1248,1904-12-25 +1904-05-05 12:34:56.1249,1904-12-25 +1904-05-05 12:34:56.125,1904-12-25 +1904-05-05 12:34:56.1251,1904-12-25 +1904-05-05 12:34:56.1252,1904-12-25 +1904-05-05 12:34:56.1253,1904-12-25 +1904-05-05 12:34:56.1254,1904-12-25 +1904-05-05 12:34:56.1255,1904-12-25 +1904-05-05 12:34:56.1256,1904-12-25 +1904-05-05 12:34:56.1257,1904-12-25 +1904-05-05 12:34:56.1258,1904-12-25 +1904-05-05 12:34:56.1259,1904-12-25 +1904-05-05 12:34:56.126,1904-12-25 +1904-05-05 12:34:56.1261,1904-12-25 +1904-05-05 12:34:56.1262,1904-12-25 +1904-05-05 12:34:56.1263,1904-12-25 +1904-05-05 12:34:56.1264,1904-12-25 +1904-05-05 12:34:56.1265,1904-12-25 +1904-05-05 12:34:56.1266,1904-12-25 +1904-05-05 12:34:56.1267,1904-12-25 +1904-05-05 12:34:56.1268,1904-12-25 +1904-05-05 12:34:56.1269,1904-12-25 +1904-05-05 12:34:56.127,1904-12-25 +1904-05-05 12:34:56.1271,1904-12-25 +1904-05-05 12:34:56.1272,1904-12-25 +1904-05-05 12:34:56.1273,1904-12-25 +1904-05-05 12:34:56.1274,1904-12-25 +1904-05-05 12:34:56.1275,1904-12-25 +1904-05-05 12:34:56.1276,1904-12-25 +1904-05-05 12:34:56.1277,1904-12-25 +1904-05-05 12:34:56.1278,1904-12-25 +1904-05-05 12:34:56.1279,1904-12-25 +1904-05-05 12:34:56.128,1904-12-25 +1904-05-05 12:34:56.1281,1904-12-25 +1904-05-05 12:34:56.1282,1904-12-25 +1904-05-05 12:34:56.1283,1904-12-25 +1904-05-05 12:34:56.1284,1904-12-25 +1904-05-05 12:34:56.1285,1904-12-25 +1904-05-05 12:34:56.1286,1904-12-25 +1904-05-05 12:34:56.1287,1904-12-25 +1904-05-05 12:34:56.1288,1904-12-25 +1904-05-05 12:34:56.1289,1904-12-25 +1904-05-05 12:34:56.129,1904-12-25 +1904-05-05 12:34:56.1291,1904-12-25 +1904-05-05 12:34:56.1292,1904-12-25 +1904-05-05 12:34:56.1293,1904-12-25 +1904-05-05 12:34:56.1294,1904-12-25 +1904-05-05 12:34:56.1295,1904-12-25 +1904-05-05 12:34:56.1296,1904-12-25 +1904-05-05 12:34:56.1297,1904-12-25 +1904-05-05 12:34:56.1298,1904-12-25 +1904-05-05 12:34:56.1299,1904-12-25 +1904-05-05 12:34:56.13,1904-12-25 +1904-05-05 12:34:56.1301,1904-12-25 +1904-05-05 12:34:56.1302,1904-12-25 +1904-05-05 12:34:56.1303,1904-12-25 +1904-05-05 12:34:56.1304,1904-12-25 +1904-05-05 12:34:56.1305,1904-12-25 +1904-05-05 12:34:56.1306,1904-12-25 +1904-05-05 12:34:56.1307,1904-12-25 +1904-05-05 12:34:56.1308,1904-12-25 +1904-05-05 12:34:56.1309,1904-12-25 +1904-05-05 12:34:56.131,1904-12-25 +1904-05-05 12:34:56.1311,1904-12-25 +1904-05-05 12:34:56.1312,1904-12-25 +1904-05-05 12:34:56.1313,1904-12-25 +1904-05-05 12:34:56.1314,1904-12-25 +1904-05-05 12:34:56.1315,1904-12-25 +1904-05-05 12:34:56.1316,1904-12-25 +1904-05-05 12:34:56.1317,1904-12-25 +1904-05-05 12:34:56.1318,1904-12-25 +1904-05-05 12:34:56.1319,1904-12-25 +1904-05-05 12:34:56.132,1904-12-25 +1904-05-05 12:34:56.1321,1904-12-25 +1904-05-05 12:34:56.1322,1904-12-25 +1904-05-05 12:34:56.1323,1904-12-25 +1904-05-05 12:34:56.1324,1904-12-25 +1904-05-05 12:34:56.1325,1904-12-25 +1904-05-05 12:34:56.1326,1904-12-25 +1904-05-05 12:34:56.1327,1904-12-25 +1904-05-05 12:34:56.1328,1904-12-25 +1904-05-05 12:34:56.1329,1904-12-25 +1904-05-05 12:34:56.133,1904-12-25 +1904-05-05 12:34:56.1331,1904-12-25 +1904-05-05 12:34:56.1332,1904-12-25 +1904-05-05 12:34:56.1333,1904-12-25 +1904-05-05 12:34:56.1334,1904-12-25 +1904-05-05 12:34:56.1335,1904-12-25 +1904-05-05 12:34:56.1336,1904-12-25 +1904-05-05 12:34:56.1337,1904-12-25 +1904-05-05 12:34:56.1338,1904-12-25 +1904-05-05 12:34:56.1339,1904-12-25 +1904-05-05 12:34:56.134,1904-12-25 +1904-05-05 12:34:56.1341,1904-12-25 +1904-05-05 12:34:56.1342,1904-12-25 +1904-05-05 12:34:56.1343,1904-12-25 +1904-05-05 12:34:56.1344,1904-12-25 +1904-05-05 12:34:56.1345,1904-12-25 +1904-05-05 12:34:56.1346,1904-12-25 +1904-05-05 12:34:56.1347,1904-12-25 +1904-05-05 12:34:56.1348,1904-12-25 +1904-05-05 12:34:56.1349,1904-12-25 +1904-05-05 12:34:56.135,1904-12-25 +1904-05-05 12:34:56.1351,1904-12-25 +1904-05-05 12:34:56.1352,1904-12-25 +1904-05-05 12:34:56.1353,1904-12-25 +1904-05-05 12:34:56.1354,1904-12-25 +1904-05-05 12:34:56.1355,1904-12-25 +1904-05-05 12:34:56.1356,1904-12-25 +1904-05-05 12:34:56.1357,1904-12-25 +1904-05-05 12:34:56.1358,1904-12-25 +1904-05-05 12:34:56.1359,1904-12-25 +1904-05-05 12:34:56.136,1904-12-25 +1904-05-05 12:34:56.1361,1904-12-25 +1904-05-05 12:34:56.1362,1904-12-25 +1904-05-05 12:34:56.1363,1904-12-25 +1904-05-05 12:34:56.1364,1904-12-25 +1904-05-05 12:34:56.1365,1904-12-25 +1904-05-05 12:34:56.1366,1904-12-25 +1904-05-05 12:34:56.1367,1904-12-25 +1904-05-05 12:34:56.1368,1904-12-25 +1904-05-05 12:34:56.1369,1904-12-25 +1904-05-05 12:34:56.137,1904-12-25 +1904-05-05 12:34:56.1371,1904-12-25 +1904-05-05 12:34:56.1372,1904-12-25 +1904-05-05 12:34:56.1373,1904-12-25 +1904-05-05 12:34:56.1374,1904-12-25 +1904-05-05 12:34:56.1375,1904-12-25 +1904-05-05 12:34:56.1376,1904-12-25 +1904-05-05 12:34:56.1377,1904-12-25 +1904-05-05 12:34:56.1378,1904-12-25 +1904-05-05 12:34:56.1379,1904-12-25 +1904-05-05 12:34:56.138,1904-12-25 +1904-05-05 12:34:56.1381,1904-12-25 +1904-05-05 12:34:56.1382,1904-12-25 +1904-05-05 12:34:56.1383,1904-12-25 +1904-05-05 12:34:56.1384,1904-12-25 +1904-05-05 12:34:56.1385,1904-12-25 +1904-05-05 12:34:56.1386,1904-12-25 +1904-05-05 12:34:56.1387,1904-12-25 +1904-05-05 12:34:56.1388,1904-12-25 +1904-05-05 12:34:56.1389,1904-12-25 +1904-05-05 12:34:56.139,1904-12-25 +1904-05-05 12:34:56.1391,1904-12-25 +1904-05-05 12:34:56.1392,1904-12-25 +1904-05-05 12:34:56.1393,1904-12-25 +1904-05-05 12:34:56.1394,1904-12-25 +1904-05-05 12:34:56.1395,1904-12-25 +1904-05-05 12:34:56.1396,1904-12-25 +1904-05-05 12:34:56.1397,1904-12-25 +1904-05-05 12:34:56.1398,1904-12-25 +1904-05-05 12:34:56.1399,1904-12-25 +1904-05-05 12:34:56.14,1904-12-25 +1904-05-05 12:34:56.1401,1904-12-25 +1904-05-05 12:34:56.1402,1904-12-25 +1904-05-05 12:34:56.1403,1904-12-25 +1904-05-05 12:34:56.1404,1904-12-25 +1904-05-05 12:34:56.1405,1904-12-25 +1904-05-05 12:34:56.1406,1904-12-25 +1904-05-05 12:34:56.1407,1904-12-25 +1904-05-05 12:34:56.1408,1904-12-25 +1904-05-05 12:34:56.1409,1904-12-25 +1904-05-05 12:34:56.141,1904-12-25 +1904-05-05 12:34:56.1411,1904-12-25 +1904-05-05 12:34:56.1412,1904-12-25 +1904-05-05 12:34:56.1413,1904-12-25 +1904-05-05 12:34:56.1414,1904-12-25 +1904-05-05 12:34:56.1415,1904-12-25 +1904-05-05 12:34:56.1416,1904-12-25 +1904-05-05 12:34:56.1417,1904-12-25 +1904-05-05 12:34:56.1418,1904-12-25 +1904-05-05 12:34:56.1419,1904-12-25 +1904-05-05 12:34:56.142,1904-12-25 +1904-05-05 12:34:56.1421,1904-12-25 +1904-05-05 12:34:56.1422,1904-12-25 +1904-05-05 12:34:56.1423,1904-12-25 +1904-05-05 12:34:56.1424,1904-12-25 +1904-05-05 12:34:56.1425,1904-12-25 +1904-05-05 12:34:56.1426,1904-12-25 +1904-05-05 12:34:56.1427,1904-12-25 +1904-05-05 12:34:56.1428,1904-12-25 +1904-05-05 12:34:56.1429,1904-12-25 +1904-05-05 12:34:56.143,1904-12-25 +1904-05-05 12:34:56.1431,1904-12-25 +1904-05-05 12:34:56.1432,1904-12-25 +1904-05-05 12:34:56.1433,1904-12-25 +1904-05-05 12:34:56.1434,1904-12-25 +1904-05-05 12:34:56.1435,1904-12-25 +1904-05-05 12:34:56.1436,1904-12-25 +1904-05-05 12:34:56.1437,1904-12-25 +1904-05-05 12:34:56.1438,1904-12-25 +1904-05-05 12:34:56.1439,1904-12-25 +1904-05-05 12:34:56.144,1904-12-25 +1904-05-05 12:34:56.1441,1904-12-25 +1904-05-05 12:34:56.1442,1904-12-25 +1904-05-05 12:34:56.1443,1904-12-25 +1904-05-05 12:34:56.1444,1904-12-25 +1904-05-05 12:34:56.1445,1904-12-25 +1904-05-05 12:34:56.1446,1904-12-25 +1904-05-05 12:34:56.1447,1904-12-25 +1904-05-05 12:34:56.1448,1904-12-25 +1904-05-05 12:34:56.1449,1904-12-25 +1904-05-05 12:34:56.145,1904-12-25 +1904-05-05 12:34:56.1451,1904-12-25 +1904-05-05 12:34:56.1452,1904-12-25 +1904-05-05 12:34:56.1453,1904-12-25 +1904-05-05 12:34:56.1454,1904-12-25 +1904-05-05 12:34:56.1455,1904-12-25 +1904-05-05 12:34:56.1456,1904-12-25 +1904-05-05 12:34:56.1457,1904-12-25 +1904-05-05 12:34:56.1458,1904-12-25 +1904-05-05 12:34:56.1459,1904-12-25 +1904-05-05 12:34:56.146,1904-12-25 +1904-05-05 12:34:56.1461,1904-12-25 +1904-05-05 12:34:56.1462,1904-12-25 +1904-05-05 12:34:56.1463,1904-12-25 +1904-05-05 12:34:56.1464,1904-12-25 +1904-05-05 12:34:56.1465,1904-12-25 +1904-05-05 12:34:56.1466,1904-12-25 +1904-05-05 12:34:56.1467,1904-12-25 +1904-05-05 12:34:56.1468,1904-12-25 +1904-05-05 12:34:56.1469,1904-12-25 +1904-05-05 12:34:56.147,1904-12-25 +1904-05-05 12:34:56.1471,1904-12-25 +1904-05-05 12:34:56.1472,1904-12-25 +1904-05-05 12:34:56.1473,1904-12-25 +1904-05-05 12:34:56.1474,1904-12-25 +1904-05-05 12:34:56.1475,1904-12-25 +1904-05-05 12:34:56.1476,1904-12-25 +1904-05-05 12:34:56.1477,1904-12-25 +1904-05-05 12:34:56.1478,1904-12-25 +1904-05-05 12:34:56.1479,1904-12-25 +1904-05-05 12:34:56.148,1904-12-25 +1904-05-05 12:34:56.1481,1904-12-25 +1904-05-05 12:34:56.1482,1904-12-25 +1904-05-05 12:34:56.1483,1904-12-25 +1904-05-05 12:34:56.1484,1904-12-25 +1904-05-05 12:34:56.1485,1904-12-25 +1904-05-05 12:34:56.1486,1904-12-25 +1904-05-05 12:34:56.1487,1904-12-25 +1904-05-05 12:34:56.1488,1904-12-25 +1904-05-05 12:34:56.1489,1904-12-25 +1904-05-05 12:34:56.149,1904-12-25 +1904-05-05 12:34:56.1491,1904-12-25 +1904-05-05 12:34:56.1492,1904-12-25 +1904-05-05 12:34:56.1493,1904-12-25 +1904-05-05 12:34:56.1494,1904-12-25 +1904-05-05 12:34:56.1495,1904-12-25 +1904-05-05 12:34:56.1496,1904-12-25 +1904-05-05 12:34:56.1497,1904-12-25 +1904-05-05 12:34:56.1498,1904-12-25 +1904-05-05 12:34:56.1499,1904-12-25 +1904-05-05 12:34:56.15,1904-12-25 +1904-05-05 12:34:56.1501,1904-12-25 +1904-05-05 12:34:56.1502,1904-12-25 +1904-05-05 12:34:56.1503,1904-12-25 +1904-05-05 12:34:56.1504,1904-12-25 +1904-05-05 12:34:56.1505,1904-12-25 +1904-05-05 12:34:56.1506,1904-12-25 +1904-05-05 12:34:56.1507,1904-12-25 +1904-05-05 12:34:56.1508,1904-12-25 +1904-05-05 12:34:56.1509,1904-12-25 +1904-05-05 12:34:56.151,1904-12-25 +1904-05-05 12:34:56.1511,1904-12-25 +1904-05-05 12:34:56.1512,1904-12-25 +1904-05-05 12:34:56.1513,1904-12-25 +1904-05-05 12:34:56.1514,1904-12-25 +1904-05-05 12:34:56.1515,1904-12-25 +1904-05-05 12:34:56.1516,1904-12-25 +1904-05-05 12:34:56.1517,1904-12-25 +1904-05-05 12:34:56.1518,1904-12-25 +1904-05-05 12:34:56.1519,1904-12-25 +1904-05-05 12:34:56.152,1904-12-25 +1904-05-05 12:34:56.1521,1904-12-25 +1904-05-05 12:34:56.1522,1904-12-25 +1904-05-05 12:34:56.1523,1904-12-25 +1904-05-05 12:34:56.1524,1904-12-25 +1904-05-05 12:34:56.1525,1904-12-25 +1904-05-05 12:34:56.1526,1904-12-25 +1904-05-05 12:34:56.1527,1904-12-25 +1904-05-05 12:34:56.1528,1904-12-25 +1904-05-05 12:34:56.1529,1904-12-25 +1904-05-05 12:34:56.153,1904-12-25 +1904-05-05 12:34:56.1531,1904-12-25 +1904-05-05 12:34:56.1532,1904-12-25 +1904-05-05 12:34:56.1533,1904-12-25 +1904-05-05 12:34:56.1534,1904-12-25 +1904-05-05 12:34:56.1535,1904-12-25 +1904-05-05 12:34:56.1536,1904-12-25 +1904-05-05 12:34:56.1537,1904-12-25 +1904-05-05 12:34:56.1538,1904-12-25 +1904-05-05 12:34:56.1539,1904-12-25 +1904-05-05 12:34:56.154,1904-12-25 +1904-05-05 12:34:56.1541,1904-12-25 +1904-05-05 12:34:56.1542,1904-12-25 +1904-05-05 12:34:56.1543,1904-12-25 +1904-05-05 12:34:56.1544,1904-12-25 +1904-05-05 12:34:56.1545,1904-12-25 +1904-05-05 12:34:56.1546,1904-12-25 +1904-05-05 12:34:56.1547,1904-12-25 +1904-05-05 12:34:56.1548,1904-12-25 +1904-05-05 12:34:56.1549,1904-12-25 +1904-05-05 12:34:56.155,1904-12-25 +1904-05-05 12:34:56.1551,1904-12-25 +1904-05-05 12:34:56.1552,1904-12-25 +1904-05-05 12:34:56.1553,1904-12-25 +1904-05-05 12:34:56.1554,1904-12-25 +1904-05-05 12:34:56.1555,1904-12-25 +1904-05-05 12:34:56.1556,1904-12-25 +1904-05-05 12:34:56.1557,1904-12-25 +1904-05-05 12:34:56.1558,1904-12-25 +1904-05-05 12:34:56.1559,1904-12-25 +1904-05-05 12:34:56.156,1904-12-25 +1904-05-05 12:34:56.1561,1904-12-25 +1904-05-05 12:34:56.1562,1904-12-25 +1904-05-05 12:34:56.1563,1904-12-25 +1904-05-05 12:34:56.1564,1904-12-25 +1904-05-05 12:34:56.1565,1904-12-25 +1904-05-05 12:34:56.1566,1904-12-25 +1904-05-05 12:34:56.1567,1904-12-25 +1904-05-05 12:34:56.1568,1904-12-25 +1904-05-05 12:34:56.1569,1904-12-25 +1904-05-05 12:34:56.157,1904-12-25 +1904-05-05 12:34:56.1571,1904-12-25 +1904-05-05 12:34:56.1572,1904-12-25 +1904-05-05 12:34:56.1573,1904-12-25 +1904-05-05 12:34:56.1574,1904-12-25 +1904-05-05 12:34:56.1575,1904-12-25 +1904-05-05 12:34:56.1576,1904-12-25 +1904-05-05 12:34:56.1577,1904-12-25 +1904-05-05 12:34:56.1578,1904-12-25 +1904-05-05 12:34:56.1579,1904-12-25 +1904-05-05 12:34:56.158,1904-12-25 +1904-05-05 12:34:56.1581,1904-12-25 +1904-05-05 12:34:56.1582,1904-12-25 +1904-05-05 12:34:56.1583,1904-12-25 +1904-05-05 12:34:56.1584,1904-12-25 +1904-05-05 12:34:56.1585,1904-12-25 +1904-05-05 12:34:56.1586,1904-12-25 +1904-05-05 12:34:56.1587,1904-12-25 +1904-05-05 12:34:56.1588,1904-12-25 +1904-05-05 12:34:56.1589,1904-12-25 +1904-05-05 12:34:56.159,1904-12-25 +1904-05-05 12:34:56.1591,1904-12-25 +1904-05-05 12:34:56.1592,1904-12-25 +1904-05-05 12:34:56.1593,1904-12-25 +1904-05-05 12:34:56.1594,1904-12-25 +1904-05-05 12:34:56.1595,1904-12-25 +1904-05-05 12:34:56.1596,1904-12-25 +1904-05-05 12:34:56.1597,1904-12-25 +1904-05-05 12:34:56.1598,1904-12-25 +1904-05-05 12:34:56.1599,1904-12-25 +1904-05-05 12:34:56.16,1904-12-25 +1904-05-05 12:34:56.1601,1904-12-25 +1904-05-05 12:34:56.1602,1904-12-25 +1904-05-05 12:34:56.1603,1904-12-25 +1904-05-05 12:34:56.1604,1904-12-25 +1904-05-05 12:34:56.1605,1904-12-25 +1904-05-05 12:34:56.1606,1904-12-25 +1904-05-05 12:34:56.1607,1904-12-25 +1904-05-05 12:34:56.1608,1904-12-25 +1904-05-05 12:34:56.1609,1904-12-25 +1904-05-05 12:34:56.161,1904-12-25 +1904-05-05 12:34:56.1611,1904-12-25 +1904-05-05 12:34:56.1612,1904-12-25 +1904-05-05 12:34:56.1613,1904-12-25 +1904-05-05 12:34:56.1614,1904-12-25 +1904-05-05 12:34:56.1615,1904-12-25 +1904-05-05 12:34:56.1616,1904-12-25 +1904-05-05 12:34:56.1617,1904-12-25 +1904-05-05 12:34:56.1618,1904-12-25 +1904-05-05 12:34:56.1619,1904-12-25 +1904-05-05 12:34:56.162,1904-12-25 +1904-05-05 12:34:56.1621,1904-12-25 +1904-05-05 12:34:56.1622,1904-12-25 +1904-05-05 12:34:56.1623,1904-12-25 +1904-05-05 12:34:56.1624,1904-12-25 +1904-05-05 12:34:56.1625,1904-12-25 +1904-05-05 12:34:56.1626,1904-12-25 +1904-05-05 12:34:56.1627,1904-12-25 +1904-05-05 12:34:56.1628,1904-12-25 +1904-05-05 12:34:56.1629,1904-12-25 +1904-05-05 12:34:56.163,1904-12-25 +1904-05-05 12:34:56.1631,1904-12-25 +1904-05-05 12:34:56.1632,1904-12-25 +1904-05-05 12:34:56.1633,1904-12-25 +1904-05-05 12:34:56.1634,1904-12-25 +1904-05-05 12:34:56.1635,1904-12-25 +1904-05-05 12:34:56.1636,1904-12-25 +1904-05-05 12:34:56.1637,1904-12-25 +1904-05-05 12:34:56.1638,1904-12-25 +1904-05-05 12:34:56.1639,1904-12-25 +1904-05-05 12:34:56.164,1904-12-25 +1904-05-05 12:34:56.1641,1904-12-25 +1904-05-05 12:34:56.1642,1904-12-25 +1904-05-05 12:34:56.1643,1904-12-25 +1904-05-05 12:34:56.1644,1904-12-25 +1904-05-05 12:34:56.1645,1904-12-25 +1904-05-05 12:34:56.1646,1904-12-25 +1904-05-05 12:34:56.1647,1904-12-25 +1904-05-05 12:34:56.1648,1904-12-25 +1904-05-05 12:34:56.1649,1904-12-25 +1904-05-05 12:34:56.165,1904-12-25 +1904-05-05 12:34:56.1651,1904-12-25 +1904-05-05 12:34:56.1652,1904-12-25 +1904-05-05 12:34:56.1653,1904-12-25 +1904-05-05 12:34:56.1654,1904-12-25 +1904-05-05 12:34:56.1655,1904-12-25 +1904-05-05 12:34:56.1656,1904-12-25 +1904-05-05 12:34:56.1657,1904-12-25 +1904-05-05 12:34:56.1658,1904-12-25 +1904-05-05 12:34:56.1659,1904-12-25 +1904-05-05 12:34:56.166,1904-12-25 +1904-05-05 12:34:56.1661,1904-12-25 +1904-05-05 12:34:56.1662,1904-12-25 +1904-05-05 12:34:56.1663,1904-12-25 +1904-05-05 12:34:56.1664,1904-12-25 +1904-05-05 12:34:56.1665,1904-12-25 +1904-05-05 12:34:56.1666,1904-12-25 +1904-05-05 12:34:56.1667,1904-12-25 +1904-05-05 12:34:56.1668,1904-12-25 +1904-05-05 12:34:56.1669,1904-12-25 +1904-05-05 12:34:56.167,1904-12-25 +1904-05-05 12:34:56.1671,1904-12-25 +1904-05-05 12:34:56.1672,1904-12-25 +1904-05-05 12:34:56.1673,1904-12-25 +1904-05-05 12:34:56.1674,1904-12-25 +1904-05-05 12:34:56.1675,1904-12-25 +1904-05-05 12:34:56.1676,1904-12-25 +1904-05-05 12:34:56.1677,1904-12-25 +1904-05-05 12:34:56.1678,1904-12-25 +1904-05-05 12:34:56.1679,1904-12-25 +1904-05-05 12:34:56.168,1904-12-25 +1904-05-05 12:34:56.1681,1904-12-25 +1904-05-05 12:34:56.1682,1904-12-25 +1904-05-05 12:34:56.1683,1904-12-25 +1904-05-05 12:34:56.1684,1904-12-25 +1904-05-05 12:34:56.1685,1904-12-25 +1904-05-05 12:34:56.1686,1904-12-25 +1904-05-05 12:34:56.1687,1904-12-25 +1904-05-05 12:34:56.1688,1904-12-25 +1904-05-05 12:34:56.1689,1904-12-25 +1904-05-05 12:34:56.169,1904-12-25 +1904-05-05 12:34:56.1691,1904-12-25 +1904-05-05 12:34:56.1692,1904-12-25 +1904-05-05 12:34:56.1693,1904-12-25 +1904-05-05 12:34:56.1694,1904-12-25 +1904-05-05 12:34:56.1695,1904-12-25 +1904-05-05 12:34:56.1696,1904-12-25 +1904-05-05 12:34:56.1697,1904-12-25 +1904-05-05 12:34:56.1698,1904-12-25 +1904-05-05 12:34:56.1699,1904-12-25 +1904-05-05 12:34:56.17,1904-12-25 +1904-05-05 12:34:56.1701,1904-12-25 +1904-05-05 12:34:56.1702,1904-12-25 +1904-05-05 12:34:56.1703,1904-12-25 +1904-05-05 12:34:56.1704,1904-12-25 +1904-05-05 12:34:56.1705,1904-12-25 +1904-05-05 12:34:56.1706,1904-12-25 +1904-05-05 12:34:56.1707,1904-12-25 +1904-05-05 12:34:56.1708,1904-12-25 +1904-05-05 12:34:56.1709,1904-12-25 +1904-05-05 12:34:56.171,1904-12-25 +1904-05-05 12:34:56.1711,1904-12-25 +1904-05-05 12:34:56.1712,1904-12-25 +1904-05-05 12:34:56.1713,1904-12-25 +1904-05-05 12:34:56.1714,1904-12-25 +1904-05-05 12:34:56.1715,1904-12-25 +1904-05-05 12:34:56.1716,1904-12-25 +1904-05-05 12:34:56.1717,1904-12-25 +1904-05-05 12:34:56.1718,1904-12-25 +1904-05-05 12:34:56.1719,1904-12-25 +1904-05-05 12:34:56.172,1904-12-25 +1904-05-05 12:34:56.1721,1904-12-25 +1904-05-05 12:34:56.1722,1904-12-25 +1904-05-05 12:34:56.1723,1904-12-25 +1904-05-05 12:34:56.1724,1904-12-25 +1904-05-05 12:34:56.1725,1904-12-25 +1904-05-05 12:34:56.1726,1904-12-25 +1904-05-05 12:34:56.1727,1904-12-25 +1904-05-05 12:34:56.1728,1904-12-25 +1904-05-05 12:34:56.1729,1904-12-25 +1904-05-05 12:34:56.173,1904-12-25 +1904-05-05 12:34:56.1731,1904-12-25 +1904-05-05 12:34:56.1732,1904-12-25 +1904-05-05 12:34:56.1733,1904-12-25 +1904-05-05 12:34:56.1734,1904-12-25 +1904-05-05 12:34:56.1735,1904-12-25 +1904-05-05 12:34:56.1736,1904-12-25 +1904-05-05 12:34:56.1737,1904-12-25 +1904-05-05 12:34:56.1738,1904-12-25 +1904-05-05 12:34:56.1739,1904-12-25 +1904-05-05 12:34:56.174,1904-12-25 +1904-05-05 12:34:56.1741,1904-12-25 +1904-05-05 12:34:56.1742,1904-12-25 +1904-05-05 12:34:56.1743,1904-12-25 +1904-05-05 12:34:56.1744,1904-12-25 +1904-05-05 12:34:56.1745,1904-12-25 +1904-05-05 12:34:56.1746,1904-12-25 +1904-05-05 12:34:56.1747,1904-12-25 +1904-05-05 12:34:56.1748,1904-12-25 +1904-05-05 12:34:56.1749,1904-12-25 +1904-05-05 12:34:56.175,1904-12-25 +1904-05-05 12:34:56.1751,1904-12-25 +1904-05-05 12:34:56.1752,1904-12-25 +1904-05-05 12:34:56.1753,1904-12-25 +1904-05-05 12:34:56.1754,1904-12-25 +1904-05-05 12:34:56.1755,1904-12-25 +1904-05-05 12:34:56.1756,1904-12-25 +1904-05-05 12:34:56.1757,1904-12-25 +1904-05-05 12:34:56.1758,1904-12-25 +1904-05-05 12:34:56.1759,1904-12-25 +1904-05-05 12:34:56.176,1904-12-25 +1904-05-05 12:34:56.1761,1904-12-25 +1904-05-05 12:34:56.1762,1904-12-25 +1904-05-05 12:34:56.1763,1904-12-25 +1904-05-05 12:34:56.1764,1904-12-25 +1904-05-05 12:34:56.1765,1904-12-25 +1904-05-05 12:34:56.1766,1904-12-25 +1904-05-05 12:34:56.1767,1904-12-25 +1904-05-05 12:34:56.1768,1904-12-25 +1904-05-05 12:34:56.1769,1904-12-25 +1904-05-05 12:34:56.177,1904-12-25 +1904-05-05 12:34:56.1771,1904-12-25 +1904-05-05 12:34:56.1772,1904-12-25 +1904-05-05 12:34:56.1773,1904-12-25 +1904-05-05 12:34:56.1774,1904-12-25 +1904-05-05 12:34:56.1775,1904-12-25 +1904-05-05 12:34:56.1776,1904-12-25 +1904-05-05 12:34:56.1777,1904-12-25 +1904-05-05 12:34:56.1778,1904-12-25 +1904-05-05 12:34:56.1779,1904-12-25 +1904-05-05 12:34:56.178,1904-12-25 +1904-05-05 12:34:56.1781,1904-12-25 +1904-05-05 12:34:56.1782,1904-12-25 +1904-05-05 12:34:56.1783,1904-12-25 +1904-05-05 12:34:56.1784,1904-12-25 +1904-05-05 12:34:56.1785,1904-12-25 +1904-05-05 12:34:56.1786,1904-12-25 +1904-05-05 12:34:56.1787,1904-12-25 +1904-05-05 12:34:56.1788,1904-12-25 +1904-05-05 12:34:56.1789,1904-12-25 +1904-05-05 12:34:56.179,1904-12-25 +1904-05-05 12:34:56.1791,1904-12-25 +1904-05-05 12:34:56.1792,1904-12-25 +1904-05-05 12:34:56.1793,1904-12-25 +1904-05-05 12:34:56.1794,1904-12-25 +1904-05-05 12:34:56.1795,1904-12-25 +1904-05-05 12:34:56.1796,1904-12-25 +1904-05-05 12:34:56.1797,1904-12-25 +1904-05-05 12:34:56.1798,1904-12-25 +1904-05-05 12:34:56.1799,1904-12-25 +1904-05-05 12:34:56.18,1904-12-25 +1904-05-05 12:34:56.1801,1904-12-25 +1904-05-05 12:34:56.1802,1904-12-25 +1904-05-05 12:34:56.1803,1904-12-25 +1904-05-05 12:34:56.1804,1904-12-25 +1904-05-05 12:34:56.1805,1904-12-25 +1904-05-05 12:34:56.1806,1904-12-25 +1904-05-05 12:34:56.1807,1904-12-25 +1904-05-05 12:34:56.1808,1904-12-25 +1904-05-05 12:34:56.1809,1904-12-25 +1904-05-05 12:34:56.181,1904-12-25 +1904-05-05 12:34:56.1811,1904-12-25 +1904-05-05 12:34:56.1812,1904-12-25 +1904-05-05 12:34:56.1813,1904-12-25 +1904-05-05 12:34:56.1814,1904-12-25 +1904-05-05 12:34:56.1815,1904-12-25 +1904-05-05 12:34:56.1816,1904-12-25 +1904-05-05 12:34:56.1817,1904-12-25 +1904-05-05 12:34:56.1818,1904-12-25 +1904-05-05 12:34:56.1819,1904-12-25 +1904-05-05 12:34:56.182,1904-12-25 +1904-05-05 12:34:56.1821,1904-12-25 +1904-05-05 12:34:56.1822,1904-12-25 +1904-05-05 12:34:56.1823,1904-12-25 +1904-05-05 12:34:56.1824,1904-12-25 +1904-05-05 12:34:56.1825,1904-12-25 +1904-05-05 12:34:56.1826,1904-12-25 +1904-05-05 12:34:56.1827,1904-12-25 +1904-05-05 12:34:56.1828,1904-12-25 +1904-05-05 12:34:56.1829,1904-12-25 +1904-05-05 12:34:56.183,1904-12-25 +1904-05-05 12:34:56.1831,1904-12-25 +1904-05-05 12:34:56.1832,1904-12-25 +1904-05-05 12:34:56.1833,1904-12-25 +1904-05-05 12:34:56.1834,1904-12-25 +1904-05-05 12:34:56.1835,1904-12-25 +1904-05-05 12:34:56.1836,1904-12-25 +1904-05-05 12:34:56.1837,1904-12-25 +1904-05-05 12:34:56.1838,1904-12-25 +1904-05-05 12:34:56.1839,1904-12-25 +1904-05-05 12:34:56.184,1904-12-25 +1904-05-05 12:34:56.1841,1904-12-25 +1904-05-05 12:34:56.1842,1904-12-25 +1904-05-05 12:34:56.1843,1904-12-25 +1904-05-05 12:34:56.1844,1904-12-25 +1904-05-05 12:34:56.1845,1904-12-25 +1904-05-05 12:34:56.1846,1904-12-25 +1904-05-05 12:34:56.1847,1904-12-25 +1904-05-05 12:34:56.1848,1904-12-25 +1904-05-05 12:34:56.1849,1904-12-25 +1904-05-05 12:34:56.185,1904-12-25 +1904-05-05 12:34:56.1851,1904-12-25 +1904-05-05 12:34:56.1852,1904-12-25 +1904-05-05 12:34:56.1853,1904-12-25 +1904-05-05 12:34:56.1854,1904-12-25 +1904-05-05 12:34:56.1855,1904-12-25 +1904-05-05 12:34:56.1856,1904-12-25 +1904-05-05 12:34:56.1857,1904-12-25 +1904-05-05 12:34:56.1858,1904-12-25 +1904-05-05 12:34:56.1859,1904-12-25 +1904-05-05 12:34:56.186,1904-12-25 +1904-05-05 12:34:56.1861,1904-12-25 +1904-05-05 12:34:56.1862,1904-12-25 +1904-05-05 12:34:56.1863,1904-12-25 +1904-05-05 12:34:56.1864,1904-12-25 +1904-05-05 12:34:56.1865,1904-12-25 +1904-05-05 12:34:56.1866,1904-12-25 +1904-05-05 12:34:56.1867,1904-12-25 +1904-05-05 12:34:56.1868,1904-12-25 +1904-05-05 12:34:56.1869,1904-12-25 +1904-05-05 12:34:56.187,1904-12-25 +1904-05-05 12:34:56.1871,1904-12-25 +1904-05-05 12:34:56.1872,1904-12-25 +1904-05-05 12:34:56.1873,1904-12-25 +1904-05-05 12:34:56.1874,1904-12-25 +1904-05-05 12:34:56.1875,1904-12-25 +1904-05-05 12:34:56.1876,1904-12-25 +1904-05-05 12:34:56.1877,1904-12-25 +1904-05-05 12:34:56.1878,1904-12-25 +1904-05-05 12:34:56.1879,1904-12-25 +1904-05-05 12:34:56.188,1904-12-25 +1904-05-05 12:34:56.1881,1904-12-25 +1904-05-05 12:34:56.1882,1904-12-25 +1904-05-05 12:34:56.1883,1904-12-25 +1904-05-05 12:34:56.1884,1904-12-25 +1904-05-05 12:34:56.1885,1904-12-25 +1904-05-05 12:34:56.1886,1904-12-25 +1904-05-05 12:34:56.1887,1904-12-25 +1904-05-05 12:34:56.1888,1904-12-25 +1904-05-05 12:34:56.1889,1904-12-25 +1904-05-05 12:34:56.189,1904-12-25 +1904-05-05 12:34:56.1891,1904-12-25 +1904-05-05 12:34:56.1892,1904-12-25 +1904-05-05 12:34:56.1893,1904-12-25 +1904-05-05 12:34:56.1894,1904-12-25 +1904-05-05 12:34:56.1895,1904-12-25 +1904-05-05 12:34:56.1896,1904-12-25 +1904-05-05 12:34:56.1897,1904-12-25 +1904-05-05 12:34:56.1898,1904-12-25 +1904-05-05 12:34:56.1899,1904-12-25 +1904-05-05 12:34:56.19,1904-12-25 +1904-05-05 12:34:56.1901,1904-12-25 +1904-05-05 12:34:56.1902,1904-12-25 +1904-05-05 12:34:56.1903,1904-12-25 +1904-05-05 12:34:56.1904,1904-12-25 +1904-05-05 12:34:56.1905,1904-12-25 +1904-05-05 12:34:56.1906,1904-12-25 +1904-05-05 12:34:56.1907,1904-12-25 +1904-05-05 12:34:56.1908,1904-12-25 +1904-05-05 12:34:56.1909,1904-12-25 +1904-05-05 12:34:56.191,1904-12-25 +1904-05-05 12:34:56.1911,1904-12-25 +1904-05-05 12:34:56.1912,1904-12-25 +1904-05-05 12:34:56.1913,1904-12-25 +1904-05-05 12:34:56.1914,1904-12-25 +1904-05-05 12:34:56.1915,1904-12-25 +1904-05-05 12:34:56.1916,1904-12-25 +1904-05-05 12:34:56.1917,1904-12-25 +1904-05-05 12:34:56.1918,1904-12-25 +1904-05-05 12:34:56.1919,1904-12-25 +1904-05-05 12:34:56.192,1904-12-25 +1904-05-05 12:34:56.1921,1904-12-25 +1904-05-05 12:34:56.1922,1904-12-25 +1904-05-05 12:34:56.1923,1904-12-25 +1904-05-05 12:34:56.1924,1904-12-25 +1904-05-05 12:34:56.1925,1904-12-25 +1904-05-05 12:34:56.1926,1904-12-25 +1904-05-05 12:34:56.1927,1904-12-25 +1904-05-05 12:34:56.1928,1904-12-25 +1904-05-05 12:34:56.1929,1904-12-25 +1904-05-05 12:34:56.193,1904-12-25 +1904-05-05 12:34:56.1931,1904-12-25 +1904-05-05 12:34:56.1932,1904-12-25 +1904-05-05 12:34:56.1933,1904-12-25 +1904-05-05 12:34:56.1934,1904-12-25 +1904-05-05 12:34:56.1935,1904-12-25 +1904-05-05 12:34:56.1936,1904-12-25 +1904-05-05 12:34:56.1937,1904-12-25 +1904-05-05 12:34:56.1938,1904-12-25 +1904-05-05 12:34:56.1939,1904-12-25 +1904-05-05 12:34:56.194,1904-12-25 +1904-05-05 12:34:56.1941,1904-12-25 +1904-05-05 12:34:56.1942,1904-12-25 +1904-05-05 12:34:56.1943,1904-12-25 +1904-05-05 12:34:56.1944,1904-12-25 +1904-05-05 12:34:56.1945,1904-12-25 +1904-05-05 12:34:56.1946,1904-12-25 +1904-05-05 12:34:56.1947,1904-12-25 +1904-05-05 12:34:56.1948,1904-12-25 +1904-05-05 12:34:56.1949,1904-12-25 +1904-05-05 12:34:56.195,1904-12-25 +1904-05-05 12:34:56.1951,1904-12-25 +1904-05-05 12:34:56.1952,1904-12-25 +1904-05-05 12:34:56.1953,1904-12-25 +1904-05-05 12:34:56.1954,1904-12-25 +1904-05-05 12:34:56.1955,1904-12-25 +1904-05-05 12:34:56.1956,1904-12-25 +1904-05-05 12:34:56.1957,1904-12-25 +1904-05-05 12:34:56.1958,1904-12-25 +1904-05-05 12:34:56.1959,1904-12-25 +1904-05-05 12:34:56.196,1904-12-25 +1904-05-05 12:34:56.1961,1904-12-25 +1904-05-05 12:34:56.1962,1904-12-25 +1904-05-05 12:34:56.1963,1904-12-25 +1904-05-05 12:34:56.1964,1904-12-25 +1904-05-05 12:34:56.1965,1904-12-25 +1904-05-05 12:34:56.1966,1904-12-25 +1904-05-05 12:34:56.1967,1904-12-25 +1904-05-05 12:34:56.1968,1904-12-25 +1904-05-05 12:34:56.1969,1904-12-25 +1904-05-05 12:34:56.197,1904-12-25 +1904-05-05 12:34:56.1971,1904-12-25 +1904-05-05 12:34:56.1972,1904-12-25 +1904-05-05 12:34:56.1973,1904-12-25 +1904-05-05 12:34:56.1974,1904-12-25 +1904-05-05 12:34:56.1975,1904-12-25 +1904-05-05 12:34:56.1976,1904-12-25 +1904-05-05 12:34:56.1977,1904-12-25 +1904-05-05 12:34:56.1978,1904-12-25 +1904-05-05 12:34:56.1979,1904-12-25 +1904-05-05 12:34:56.198,1904-12-25 +1904-05-05 12:34:56.1981,1904-12-25 +1904-05-05 12:34:56.1982,1904-12-25 +1904-05-05 12:34:56.1983,1904-12-25 +1904-05-05 12:34:56.1984,1904-12-25 +1904-05-05 12:34:56.1985,1904-12-25 +1904-05-05 12:34:56.1986,1904-12-25 +1904-05-05 12:34:56.1987,1904-12-25 +1904-05-05 12:34:56.1988,1904-12-25 +1904-05-05 12:34:56.1989,1904-12-25 +1904-05-05 12:34:56.199,1904-12-25 +1904-05-05 12:34:56.1991,1904-12-25 +1904-05-05 12:34:56.1992,1904-12-25 +1904-05-05 12:34:56.1993,1904-12-25 +1904-05-05 12:34:56.1994,1904-12-25 +1904-05-05 12:34:56.1995,1904-12-25 +1904-05-05 12:34:56.1996,1904-12-25 +1904-05-05 12:34:56.1997,1904-12-25 +1904-05-05 12:34:56.1998,1904-12-25 +1904-05-05 12:34:56.1999,1904-12-25 +1905-05-05 12:34:56.1,1905-12-25 +1905-05-05 12:34:56.1001,1905-12-25 +1905-05-05 12:34:56.1002,1905-12-25 +1905-05-05 12:34:56.1003,1905-12-25 +1905-05-05 12:34:56.1004,1905-12-25 +1905-05-05 12:34:56.1005,1905-12-25 +1905-05-05 12:34:56.1006,1905-12-25 +1905-05-05 12:34:56.1007,1905-12-25 +1905-05-05 12:34:56.1008,1905-12-25 +1905-05-05 12:34:56.1009,1905-12-25 +1905-05-05 12:34:56.101,1905-12-25 +1905-05-05 12:34:56.1011,1905-12-25 +1905-05-05 12:34:56.1012,1905-12-25 +1905-05-05 12:34:56.1013,1905-12-25 +1905-05-05 12:34:56.1014,1905-12-25 +1905-05-05 12:34:56.1015,1905-12-25 +1905-05-05 12:34:56.1016,1905-12-25 +1905-05-05 12:34:56.1017,1905-12-25 +1905-05-05 12:34:56.1018,1905-12-25 +1905-05-05 12:34:56.1019,1905-12-25 +1905-05-05 12:34:56.102,1905-12-25 +1905-05-05 12:34:56.1021,1905-12-25 +1905-05-05 12:34:56.1022,1905-12-25 +1905-05-05 12:34:56.1023,1905-12-25 +1905-05-05 12:34:56.1024,1905-12-25 +1905-05-05 12:34:56.1025,1905-12-25 +1905-05-05 12:34:56.1026,1905-12-25 +1905-05-05 12:34:56.1027,1905-12-25 +1905-05-05 12:34:56.1028,1905-12-25 +1905-05-05 12:34:56.1029,1905-12-25 +1905-05-05 12:34:56.103,1905-12-25 +1905-05-05 12:34:56.1031,1905-12-25 +1905-05-05 12:34:56.1032,1905-12-25 +1905-05-05 12:34:56.1033,1905-12-25 +1905-05-05 12:34:56.1034,1905-12-25 +1905-05-05 12:34:56.1035,1905-12-25 +1905-05-05 12:34:56.1036,1905-12-25 +1905-05-05 12:34:56.1037,1905-12-25 +1905-05-05 12:34:56.1038,1905-12-25 +1905-05-05 12:34:56.1039,1905-12-25 +1905-05-05 12:34:56.104,1905-12-25 +1905-05-05 12:34:56.1041,1905-12-25 +1905-05-05 12:34:56.1042,1905-12-25 +1905-05-05 12:34:56.1043,1905-12-25 +1905-05-05 12:34:56.1044,1905-12-25 +1905-05-05 12:34:56.1045,1905-12-25 +1905-05-05 12:34:56.1046,1905-12-25 +1905-05-05 12:34:56.1047,1905-12-25 +1905-05-05 12:34:56.1048,1905-12-25 +1905-05-05 12:34:56.1049,1905-12-25 +1905-05-05 12:34:56.105,1905-12-25 +1905-05-05 12:34:56.1051,1905-12-25 +1905-05-05 12:34:56.1052,1905-12-25 +1905-05-05 12:34:56.1053,1905-12-25 +1905-05-05 12:34:56.1054,1905-12-25 +1905-05-05 12:34:56.1055,1905-12-25 +1905-05-05 12:34:56.1056,1905-12-25 +1905-05-05 12:34:56.1057,1905-12-25 +1905-05-05 12:34:56.1058,1905-12-25 +1905-05-05 12:34:56.1059,1905-12-25 +1905-05-05 12:34:56.106,1905-12-25 +1905-05-05 12:34:56.1061,1905-12-25 +1905-05-05 12:34:56.1062,1905-12-25 +1905-05-05 12:34:56.1063,1905-12-25 +1905-05-05 12:34:56.1064,1905-12-25 +1905-05-05 12:34:56.1065,1905-12-25 +1905-05-05 12:34:56.1066,1905-12-25 +1905-05-05 12:34:56.1067,1905-12-25 +1905-05-05 12:34:56.1068,1905-12-25 +1905-05-05 12:34:56.1069,1905-12-25 +1905-05-05 12:34:56.107,1905-12-25 +1905-05-05 12:34:56.1071,1905-12-25 +1905-05-05 12:34:56.1072,1905-12-25 +1905-05-05 12:34:56.1073,1905-12-25 +1905-05-05 12:34:56.1074,1905-12-25 +1905-05-05 12:34:56.1075,1905-12-25 +1905-05-05 12:34:56.1076,1905-12-25 +1905-05-05 12:34:56.1077,1905-12-25 +1905-05-05 12:34:56.1078,1905-12-25 +1905-05-05 12:34:56.1079,1905-12-25 +1905-05-05 12:34:56.108,1905-12-25 +1905-05-05 12:34:56.1081,1905-12-25 +1905-05-05 12:34:56.1082,1905-12-25 +1905-05-05 12:34:56.1083,1905-12-25 +1905-05-05 12:34:56.1084,1905-12-25 +1905-05-05 12:34:56.1085,1905-12-25 +1905-05-05 12:34:56.1086,1905-12-25 +1905-05-05 12:34:56.1087,1905-12-25 +1905-05-05 12:34:56.1088,1905-12-25 +1905-05-05 12:34:56.1089,1905-12-25 +1905-05-05 12:34:56.109,1905-12-25 +1905-05-05 12:34:56.1091,1905-12-25 +1905-05-05 12:34:56.1092,1905-12-25 +1905-05-05 12:34:56.1093,1905-12-25 +1905-05-05 12:34:56.1094,1905-12-25 +1905-05-05 12:34:56.1095,1905-12-25 +1905-05-05 12:34:56.1096,1905-12-25 +1905-05-05 12:34:56.1097,1905-12-25 +1905-05-05 12:34:56.1098,1905-12-25 +1905-05-05 12:34:56.1099,1905-12-25 +1905-05-05 12:34:56.11,1905-12-25 +1905-05-05 12:34:56.1101,1905-12-25 +1905-05-05 12:34:56.1102,1905-12-25 +1905-05-05 12:34:56.1103,1905-12-25 +1905-05-05 12:34:56.1104,1905-12-25 +1905-05-05 12:34:56.1105,1905-12-25 +1905-05-05 12:34:56.1106,1905-12-25 +1905-05-05 12:34:56.1107,1905-12-25 +1905-05-05 12:34:56.1108,1905-12-25 +1905-05-05 12:34:56.1109,1905-12-25 +1905-05-05 12:34:56.111,1905-12-25 +1905-05-05 12:34:56.1111,1905-12-25 +1905-05-05 12:34:56.1112,1905-12-25 +1905-05-05 12:34:56.1113,1905-12-25 +1905-05-05 12:34:56.1114,1905-12-25 +1905-05-05 12:34:56.1115,1905-12-25 +1905-05-05 12:34:56.1116,1905-12-25 +1905-05-05 12:34:56.1117,1905-12-25 +1905-05-05 12:34:56.1118,1905-12-25 +1905-05-05 12:34:56.1119,1905-12-25 +1905-05-05 12:34:56.112,1905-12-25 +1905-05-05 12:34:56.1121,1905-12-25 +1905-05-05 12:34:56.1122,1905-12-25 +1905-05-05 12:34:56.1123,1905-12-25 +1905-05-05 12:34:56.1124,1905-12-25 +1905-05-05 12:34:56.1125,1905-12-25 +1905-05-05 12:34:56.1126,1905-12-25 +1905-05-05 12:34:56.1127,1905-12-25 +1905-05-05 12:34:56.1128,1905-12-25 +1905-05-05 12:34:56.1129,1905-12-25 +1905-05-05 12:34:56.113,1905-12-25 +1905-05-05 12:34:56.1131,1905-12-25 +1905-05-05 12:34:56.1132,1905-12-25 +1905-05-05 12:34:56.1133,1905-12-25 +1905-05-05 12:34:56.1134,1905-12-25 +1905-05-05 12:34:56.1135,1905-12-25 +1905-05-05 12:34:56.1136,1905-12-25 +1905-05-05 12:34:56.1137,1905-12-25 +1905-05-05 12:34:56.1138,1905-12-25 +1905-05-05 12:34:56.1139,1905-12-25 +1905-05-05 12:34:56.114,1905-12-25 +1905-05-05 12:34:56.1141,1905-12-25 +1905-05-05 12:34:56.1142,1905-12-25 +1905-05-05 12:34:56.1143,1905-12-25 +1905-05-05 12:34:56.1144,1905-12-25 +1905-05-05 12:34:56.1145,1905-12-25 +1905-05-05 12:34:56.1146,1905-12-25 +1905-05-05 12:34:56.1147,1905-12-25 +1905-05-05 12:34:56.1148,1905-12-25 +1905-05-05 12:34:56.1149,1905-12-25 +1905-05-05 12:34:56.115,1905-12-25 +1905-05-05 12:34:56.1151,1905-12-25 +1905-05-05 12:34:56.1152,1905-12-25 +1905-05-05 12:34:56.1153,1905-12-25 +1905-05-05 12:34:56.1154,1905-12-25 +1905-05-05 12:34:56.1155,1905-12-25 +1905-05-05 12:34:56.1156,1905-12-25 +1905-05-05 12:34:56.1157,1905-12-25 +1905-05-05 12:34:56.1158,1905-12-25 +1905-05-05 12:34:56.1159,1905-12-25 +1905-05-05 12:34:56.116,1905-12-25 +1905-05-05 12:34:56.1161,1905-12-25 +1905-05-05 12:34:56.1162,1905-12-25 +1905-05-05 12:34:56.1163,1905-12-25 +1905-05-05 12:34:56.1164,1905-12-25 +1905-05-05 12:34:56.1165,1905-12-25 +1905-05-05 12:34:56.1166,1905-12-25 +1905-05-05 12:34:56.1167,1905-12-25 +1905-05-05 12:34:56.1168,1905-12-25 +1905-05-05 12:34:56.1169,1905-12-25 +1905-05-05 12:34:56.117,1905-12-25 +1905-05-05 12:34:56.1171,1905-12-25 +1905-05-05 12:34:56.1172,1905-12-25 +1905-05-05 12:34:56.1173,1905-12-25 +1905-05-05 12:34:56.1174,1905-12-25 +1905-05-05 12:34:56.1175,1905-12-25 +1905-05-05 12:34:56.1176,1905-12-25 +1905-05-05 12:34:56.1177,1905-12-25 +1905-05-05 12:34:56.1178,1905-12-25 +1905-05-05 12:34:56.1179,1905-12-25 +1905-05-05 12:34:56.118,1905-12-25 +1905-05-05 12:34:56.1181,1905-12-25 +1905-05-05 12:34:56.1182,1905-12-25 +1905-05-05 12:34:56.1183,1905-12-25 +1905-05-05 12:34:56.1184,1905-12-25 +1905-05-05 12:34:56.1185,1905-12-25 +1905-05-05 12:34:56.1186,1905-12-25 +1905-05-05 12:34:56.1187,1905-12-25 +1905-05-05 12:34:56.1188,1905-12-25 +1905-05-05 12:34:56.1189,1905-12-25 +1905-05-05 12:34:56.119,1905-12-25 +1905-05-05 12:34:56.1191,1905-12-25 +1905-05-05 12:34:56.1192,1905-12-25 +1905-05-05 12:34:56.1193,1905-12-25 +1905-05-05 12:34:56.1194,1905-12-25 +1905-05-05 12:34:56.1195,1905-12-25 +1905-05-05 12:34:56.1196,1905-12-25 +1905-05-05 12:34:56.1197,1905-12-25 +1905-05-05 12:34:56.1198,1905-12-25 +1905-05-05 12:34:56.1199,1905-12-25 +1905-05-05 12:34:56.12,1905-12-25 +1905-05-05 12:34:56.1201,1905-12-25 +1905-05-05 12:34:56.1202,1905-12-25 +1905-05-05 12:34:56.1203,1905-12-25 +1905-05-05 12:34:56.1204,1905-12-25 +1905-05-05 12:34:56.1205,1905-12-25 +1905-05-05 12:34:56.1206,1905-12-25 +1905-05-05 12:34:56.1207,1905-12-25 +1905-05-05 12:34:56.1208,1905-12-25 +1905-05-05 12:34:56.1209,1905-12-25 +1905-05-05 12:34:56.121,1905-12-25 +1905-05-05 12:34:56.1211,1905-12-25 +1905-05-05 12:34:56.1212,1905-12-25 +1905-05-05 12:34:56.1213,1905-12-25 +1905-05-05 12:34:56.1214,1905-12-25 +1905-05-05 12:34:56.1215,1905-12-25 +1905-05-05 12:34:56.1216,1905-12-25 +1905-05-05 12:34:56.1217,1905-12-25 +1905-05-05 12:34:56.1218,1905-12-25 +1905-05-05 12:34:56.1219,1905-12-25 +1905-05-05 12:34:56.122,1905-12-25 +1905-05-05 12:34:56.1221,1905-12-25 +1905-05-05 12:34:56.1222,1905-12-25 +1905-05-05 12:34:56.1223,1905-12-25 +1905-05-05 12:34:56.1224,1905-12-25 +1905-05-05 12:34:56.1225,1905-12-25 +1905-05-05 12:34:56.1226,1905-12-25 +1905-05-05 12:34:56.1227,1905-12-25 +1905-05-05 12:34:56.1228,1905-12-25 +1905-05-05 12:34:56.1229,1905-12-25 +1905-05-05 12:34:56.123,1905-12-25 +1905-05-05 12:34:56.1231,1905-12-25 +1905-05-05 12:34:56.1232,1905-12-25 +1905-05-05 12:34:56.1233,1905-12-25 +1905-05-05 12:34:56.1234,1905-12-25 +1905-05-05 12:34:56.1235,1905-12-25 +1905-05-05 12:34:56.1236,1905-12-25 +1905-05-05 12:34:56.1237,1905-12-25 +1905-05-05 12:34:56.1238,1905-12-25 +1905-05-05 12:34:56.1239,1905-12-25 +1905-05-05 12:34:56.124,1905-12-25 +1905-05-05 12:34:56.1241,1905-12-25 +1905-05-05 12:34:56.1242,1905-12-25 +1905-05-05 12:34:56.1243,1905-12-25 +1905-05-05 12:34:56.1244,1905-12-25 +1905-05-05 12:34:56.1245,1905-12-25 +1905-05-05 12:34:56.1246,1905-12-25 +1905-05-05 12:34:56.1247,1905-12-25 +1905-05-05 12:34:56.1248,1905-12-25 +1905-05-05 12:34:56.1249,1905-12-25 +1905-05-05 12:34:56.125,1905-12-25 +1905-05-05 12:34:56.1251,1905-12-25 +1905-05-05 12:34:56.1252,1905-12-25 +1905-05-05 12:34:56.1253,1905-12-25 +1905-05-05 12:34:56.1254,1905-12-25 +1905-05-05 12:34:56.1255,1905-12-25 +1905-05-05 12:34:56.1256,1905-12-25 +1905-05-05 12:34:56.1257,1905-12-25 +1905-05-05 12:34:56.1258,1905-12-25 +1905-05-05 12:34:56.1259,1905-12-25 +1905-05-05 12:34:56.126,1905-12-25 +1905-05-05 12:34:56.1261,1905-12-25 +1905-05-05 12:34:56.1262,1905-12-25 +1905-05-05 12:34:56.1263,1905-12-25 +1905-05-05 12:34:56.1264,1905-12-25 +1905-05-05 12:34:56.1265,1905-12-25 +1905-05-05 12:34:56.1266,1905-12-25 +1905-05-05 12:34:56.1267,1905-12-25 +1905-05-05 12:34:56.1268,1905-12-25 +1905-05-05 12:34:56.1269,1905-12-25 +1905-05-05 12:34:56.127,1905-12-25 +1905-05-05 12:34:56.1271,1905-12-25 +1905-05-05 12:34:56.1272,1905-12-25 +1905-05-05 12:34:56.1273,1905-12-25 +1905-05-05 12:34:56.1274,1905-12-25 +1905-05-05 12:34:56.1275,1905-12-25 +1905-05-05 12:34:56.1276,1905-12-25 +1905-05-05 12:34:56.1277,1905-12-25 +1905-05-05 12:34:56.1278,1905-12-25 +1905-05-05 12:34:56.1279,1905-12-25 +1905-05-05 12:34:56.128,1905-12-25 +1905-05-05 12:34:56.1281,1905-12-25 +1905-05-05 12:34:56.1282,1905-12-25 +1905-05-05 12:34:56.1283,1905-12-25 +1905-05-05 12:34:56.1284,1905-12-25 +1905-05-05 12:34:56.1285,1905-12-25 +1905-05-05 12:34:56.1286,1905-12-25 +1905-05-05 12:34:56.1287,1905-12-25 +1905-05-05 12:34:56.1288,1905-12-25 +1905-05-05 12:34:56.1289,1905-12-25 +1905-05-05 12:34:56.129,1905-12-25 +1905-05-05 12:34:56.1291,1905-12-25 +1905-05-05 12:34:56.1292,1905-12-25 +1905-05-05 12:34:56.1293,1905-12-25 +1905-05-05 12:34:56.1294,1905-12-25 +1905-05-05 12:34:56.1295,1905-12-25 +1905-05-05 12:34:56.1296,1905-12-25 +1905-05-05 12:34:56.1297,1905-12-25 +1905-05-05 12:34:56.1298,1905-12-25 +1905-05-05 12:34:56.1299,1905-12-25 +1905-05-05 12:34:56.13,1905-12-25 +1905-05-05 12:34:56.1301,1905-12-25 +1905-05-05 12:34:56.1302,1905-12-25 +1905-05-05 12:34:56.1303,1905-12-25 +1905-05-05 12:34:56.1304,1905-12-25 +1905-05-05 12:34:56.1305,1905-12-25 +1905-05-05 12:34:56.1306,1905-12-25 +1905-05-05 12:34:56.1307,1905-12-25 +1905-05-05 12:34:56.1308,1905-12-25 +1905-05-05 12:34:56.1309,1905-12-25 +1905-05-05 12:34:56.131,1905-12-25 +1905-05-05 12:34:56.1311,1905-12-25 +1905-05-05 12:34:56.1312,1905-12-25 +1905-05-05 12:34:56.1313,1905-12-25 +1905-05-05 12:34:56.1314,1905-12-25 +1905-05-05 12:34:56.1315,1905-12-25 +1905-05-05 12:34:56.1316,1905-12-25 +1905-05-05 12:34:56.1317,1905-12-25 +1905-05-05 12:34:56.1318,1905-12-25 +1905-05-05 12:34:56.1319,1905-12-25 +1905-05-05 12:34:56.132,1905-12-25 +1905-05-05 12:34:56.1321,1905-12-25 +1905-05-05 12:34:56.1322,1905-12-25 +1905-05-05 12:34:56.1323,1905-12-25 +1905-05-05 12:34:56.1324,1905-12-25 +1905-05-05 12:34:56.1325,1905-12-25 +1905-05-05 12:34:56.1326,1905-12-25 +1905-05-05 12:34:56.1327,1905-12-25 +1905-05-05 12:34:56.1328,1905-12-25 +1905-05-05 12:34:56.1329,1905-12-25 +1905-05-05 12:34:56.133,1905-12-25 +1905-05-05 12:34:56.1331,1905-12-25 +1905-05-05 12:34:56.1332,1905-12-25 +1905-05-05 12:34:56.1333,1905-12-25 +1905-05-05 12:34:56.1334,1905-12-25 +1905-05-05 12:34:56.1335,1905-12-25 +1905-05-05 12:34:56.1336,1905-12-25 +1905-05-05 12:34:56.1337,1905-12-25 +1905-05-05 12:34:56.1338,1905-12-25 +1905-05-05 12:34:56.1339,1905-12-25 +1905-05-05 12:34:56.134,1905-12-25 +1905-05-05 12:34:56.1341,1905-12-25 +1905-05-05 12:34:56.1342,1905-12-25 +1905-05-05 12:34:56.1343,1905-12-25 +1905-05-05 12:34:56.1344,1905-12-25 +1905-05-05 12:34:56.1345,1905-12-25 +1905-05-05 12:34:56.1346,1905-12-25 +1905-05-05 12:34:56.1347,1905-12-25 +1905-05-05 12:34:56.1348,1905-12-25 +1905-05-05 12:34:56.1349,1905-12-25 +1905-05-05 12:34:56.135,1905-12-25 +1905-05-05 12:34:56.1351,1905-12-25 +1905-05-05 12:34:56.1352,1905-12-25 +1905-05-05 12:34:56.1353,1905-12-25 +1905-05-05 12:34:56.1354,1905-12-25 +1905-05-05 12:34:56.1355,1905-12-25 +1905-05-05 12:34:56.1356,1905-12-25 +1905-05-05 12:34:56.1357,1905-12-25 +1905-05-05 12:34:56.1358,1905-12-25 +1905-05-05 12:34:56.1359,1905-12-25 +1905-05-05 12:34:56.136,1905-12-25 +1905-05-05 12:34:56.1361,1905-12-25 +1905-05-05 12:34:56.1362,1905-12-25 +1905-05-05 12:34:56.1363,1905-12-25 +1905-05-05 12:34:56.1364,1905-12-25 +1905-05-05 12:34:56.1365,1905-12-25 +1905-05-05 12:34:56.1366,1905-12-25 +1905-05-05 12:34:56.1367,1905-12-25 +1905-05-05 12:34:56.1368,1905-12-25 +1905-05-05 12:34:56.1369,1905-12-25 +1905-05-05 12:34:56.137,1905-12-25 +1905-05-05 12:34:56.1371,1905-12-25 +1905-05-05 12:34:56.1372,1905-12-25 +1905-05-05 12:34:56.1373,1905-12-25 +1905-05-05 12:34:56.1374,1905-12-25 +1905-05-05 12:34:56.1375,1905-12-25 +1905-05-05 12:34:56.1376,1905-12-25 +1905-05-05 12:34:56.1377,1905-12-25 +1905-05-05 12:34:56.1378,1905-12-25 +1905-05-05 12:34:56.1379,1905-12-25 +1905-05-05 12:34:56.138,1905-12-25 +1905-05-05 12:34:56.1381,1905-12-25 +1905-05-05 12:34:56.1382,1905-12-25 +1905-05-05 12:34:56.1383,1905-12-25 +1905-05-05 12:34:56.1384,1905-12-25 +1905-05-05 12:34:56.1385,1905-12-25 +1905-05-05 12:34:56.1386,1905-12-25 +1905-05-05 12:34:56.1387,1905-12-25 +1905-05-05 12:34:56.1388,1905-12-25 +1905-05-05 12:34:56.1389,1905-12-25 +1905-05-05 12:34:56.139,1905-12-25 +1905-05-05 12:34:56.1391,1905-12-25 +1905-05-05 12:34:56.1392,1905-12-25 +1905-05-05 12:34:56.1393,1905-12-25 +1905-05-05 12:34:56.1394,1905-12-25 +1905-05-05 12:34:56.1395,1905-12-25 +1905-05-05 12:34:56.1396,1905-12-25 +1905-05-05 12:34:56.1397,1905-12-25 +1905-05-05 12:34:56.1398,1905-12-25 +1905-05-05 12:34:56.1399,1905-12-25 +1905-05-05 12:34:56.14,1905-12-25 +1905-05-05 12:34:56.1401,1905-12-25 +1905-05-05 12:34:56.1402,1905-12-25 +1905-05-05 12:34:56.1403,1905-12-25 +1905-05-05 12:34:56.1404,1905-12-25 +1905-05-05 12:34:56.1405,1905-12-25 +1905-05-05 12:34:56.1406,1905-12-25 +1905-05-05 12:34:56.1407,1905-12-25 +1905-05-05 12:34:56.1408,1905-12-25 +1905-05-05 12:34:56.1409,1905-12-25 +1905-05-05 12:34:56.141,1905-12-25 +1905-05-05 12:34:56.1411,1905-12-25 +1905-05-05 12:34:56.1412,1905-12-25 +1905-05-05 12:34:56.1413,1905-12-25 +1905-05-05 12:34:56.1414,1905-12-25 +1905-05-05 12:34:56.1415,1905-12-25 +1905-05-05 12:34:56.1416,1905-12-25 +1905-05-05 12:34:56.1417,1905-12-25 +1905-05-05 12:34:56.1418,1905-12-25 +1905-05-05 12:34:56.1419,1905-12-25 +1905-05-05 12:34:56.142,1905-12-25 +1905-05-05 12:34:56.1421,1905-12-25 +1905-05-05 12:34:56.1422,1905-12-25 +1905-05-05 12:34:56.1423,1905-12-25 +1905-05-05 12:34:56.1424,1905-12-25 +1905-05-05 12:34:56.1425,1905-12-25 +1905-05-05 12:34:56.1426,1905-12-25 +1905-05-05 12:34:56.1427,1905-12-25 +1905-05-05 12:34:56.1428,1905-12-25 +1905-05-05 12:34:56.1429,1905-12-25 +1905-05-05 12:34:56.143,1905-12-25 +1905-05-05 12:34:56.1431,1905-12-25 +1905-05-05 12:34:56.1432,1905-12-25 +1905-05-05 12:34:56.1433,1905-12-25 +1905-05-05 12:34:56.1434,1905-12-25 +1905-05-05 12:34:56.1435,1905-12-25 +1905-05-05 12:34:56.1436,1905-12-25 +1905-05-05 12:34:56.1437,1905-12-25 +1905-05-05 12:34:56.1438,1905-12-25 +1905-05-05 12:34:56.1439,1905-12-25 +1905-05-05 12:34:56.144,1905-12-25 +1905-05-05 12:34:56.1441,1905-12-25 +1905-05-05 12:34:56.1442,1905-12-25 +1905-05-05 12:34:56.1443,1905-12-25 +1905-05-05 12:34:56.1444,1905-12-25 +1905-05-05 12:34:56.1445,1905-12-25 +1905-05-05 12:34:56.1446,1905-12-25 +1905-05-05 12:34:56.1447,1905-12-25 +1905-05-05 12:34:56.1448,1905-12-25 +1905-05-05 12:34:56.1449,1905-12-25 +1905-05-05 12:34:56.145,1905-12-25 +1905-05-05 12:34:56.1451,1905-12-25 +1905-05-05 12:34:56.1452,1905-12-25 +1905-05-05 12:34:56.1453,1905-12-25 +1905-05-05 12:34:56.1454,1905-12-25 +1905-05-05 12:34:56.1455,1905-12-25 +1905-05-05 12:34:56.1456,1905-12-25 +1905-05-05 12:34:56.1457,1905-12-25 +1905-05-05 12:34:56.1458,1905-12-25 +1905-05-05 12:34:56.1459,1905-12-25 +1905-05-05 12:34:56.146,1905-12-25 +1905-05-05 12:34:56.1461,1905-12-25 +1905-05-05 12:34:56.1462,1905-12-25 +1905-05-05 12:34:56.1463,1905-12-25 +1905-05-05 12:34:56.1464,1905-12-25 +1905-05-05 12:34:56.1465,1905-12-25 +1905-05-05 12:34:56.1466,1905-12-25 +1905-05-05 12:34:56.1467,1905-12-25 +1905-05-05 12:34:56.1468,1905-12-25 +1905-05-05 12:34:56.1469,1905-12-25 +1905-05-05 12:34:56.147,1905-12-25 +1905-05-05 12:34:56.1471,1905-12-25 +1905-05-05 12:34:56.1472,1905-12-25 +1905-05-05 12:34:56.1473,1905-12-25 +1905-05-05 12:34:56.1474,1905-12-25 +1905-05-05 12:34:56.1475,1905-12-25 +1905-05-05 12:34:56.1476,1905-12-25 +1905-05-05 12:34:56.1477,1905-12-25 +1905-05-05 12:34:56.1478,1905-12-25 +1905-05-05 12:34:56.1479,1905-12-25 +1905-05-05 12:34:56.148,1905-12-25 +1905-05-05 12:34:56.1481,1905-12-25 +1905-05-05 12:34:56.1482,1905-12-25 +1905-05-05 12:34:56.1483,1905-12-25 +1905-05-05 12:34:56.1484,1905-12-25 +1905-05-05 12:34:56.1485,1905-12-25 +1905-05-05 12:34:56.1486,1905-12-25 +1905-05-05 12:34:56.1487,1905-12-25 +1905-05-05 12:34:56.1488,1905-12-25 +1905-05-05 12:34:56.1489,1905-12-25 +1905-05-05 12:34:56.149,1905-12-25 +1905-05-05 12:34:56.1491,1905-12-25 +1905-05-05 12:34:56.1492,1905-12-25 +1905-05-05 12:34:56.1493,1905-12-25 +1905-05-05 12:34:56.1494,1905-12-25 +1905-05-05 12:34:56.1495,1905-12-25 +1905-05-05 12:34:56.1496,1905-12-25 +1905-05-05 12:34:56.1497,1905-12-25 +1905-05-05 12:34:56.1498,1905-12-25 +1905-05-05 12:34:56.1499,1905-12-25 +1905-05-05 12:34:56.15,1905-12-25 +1905-05-05 12:34:56.1501,1905-12-25 +1905-05-05 12:34:56.1502,1905-12-25 +1905-05-05 12:34:56.1503,1905-12-25 +1905-05-05 12:34:56.1504,1905-12-25 +1905-05-05 12:34:56.1505,1905-12-25 +1905-05-05 12:34:56.1506,1905-12-25 +1905-05-05 12:34:56.1507,1905-12-25 +1905-05-05 12:34:56.1508,1905-12-25 +1905-05-05 12:34:56.1509,1905-12-25 +1905-05-05 12:34:56.151,1905-12-25 +1905-05-05 12:34:56.1511,1905-12-25 +1905-05-05 12:34:56.1512,1905-12-25 +1905-05-05 12:34:56.1513,1905-12-25 +1905-05-05 12:34:56.1514,1905-12-25 +1905-05-05 12:34:56.1515,1905-12-25 +1905-05-05 12:34:56.1516,1905-12-25 +1905-05-05 12:34:56.1517,1905-12-25 +1905-05-05 12:34:56.1518,1905-12-25 +1905-05-05 12:34:56.1519,1905-12-25 +1905-05-05 12:34:56.152,1905-12-25 +1905-05-05 12:34:56.1521,1905-12-25 +1905-05-05 12:34:56.1522,1905-12-25 +1905-05-05 12:34:56.1523,1905-12-25 +1905-05-05 12:34:56.1524,1905-12-25 +1905-05-05 12:34:56.1525,1905-12-25 +1905-05-05 12:34:56.1526,1905-12-25 +1905-05-05 12:34:56.1527,1905-12-25 +1905-05-05 12:34:56.1528,1905-12-25 +1905-05-05 12:34:56.1529,1905-12-25 +1905-05-05 12:34:56.153,1905-12-25 +1905-05-05 12:34:56.1531,1905-12-25 +1905-05-05 12:34:56.1532,1905-12-25 +1905-05-05 12:34:56.1533,1905-12-25 +1905-05-05 12:34:56.1534,1905-12-25 +1905-05-05 12:34:56.1535,1905-12-25 +1905-05-05 12:34:56.1536,1905-12-25 +1905-05-05 12:34:56.1537,1905-12-25 +1905-05-05 12:34:56.1538,1905-12-25 +1905-05-05 12:34:56.1539,1905-12-25 +1905-05-05 12:34:56.154,1905-12-25 +1905-05-05 12:34:56.1541,1905-12-25 +1905-05-05 12:34:56.1542,1905-12-25 +1905-05-05 12:34:56.1543,1905-12-25 +1905-05-05 12:34:56.1544,1905-12-25 +1905-05-05 12:34:56.1545,1905-12-25 +1905-05-05 12:34:56.1546,1905-12-25 +1905-05-05 12:34:56.1547,1905-12-25 +1905-05-05 12:34:56.1548,1905-12-25 +1905-05-05 12:34:56.1549,1905-12-25 +1905-05-05 12:34:56.155,1905-12-25 +1905-05-05 12:34:56.1551,1905-12-25 +1905-05-05 12:34:56.1552,1905-12-25 +1905-05-05 12:34:56.1553,1905-12-25 +1905-05-05 12:34:56.1554,1905-12-25 +1905-05-05 12:34:56.1555,1905-12-25 +1905-05-05 12:34:56.1556,1905-12-25 +1905-05-05 12:34:56.1557,1905-12-25 +1905-05-05 12:34:56.1558,1905-12-25 +1905-05-05 12:34:56.1559,1905-12-25 +1905-05-05 12:34:56.156,1905-12-25 +1905-05-05 12:34:56.1561,1905-12-25 +1905-05-05 12:34:56.1562,1905-12-25 +1905-05-05 12:34:56.1563,1905-12-25 +1905-05-05 12:34:56.1564,1905-12-25 +1905-05-05 12:34:56.1565,1905-12-25 +1905-05-05 12:34:56.1566,1905-12-25 +1905-05-05 12:34:56.1567,1905-12-25 +1905-05-05 12:34:56.1568,1905-12-25 +1905-05-05 12:34:56.1569,1905-12-25 +1905-05-05 12:34:56.157,1905-12-25 +1905-05-05 12:34:56.1571,1905-12-25 +1905-05-05 12:34:56.1572,1905-12-25 +1905-05-05 12:34:56.1573,1905-12-25 +1905-05-05 12:34:56.1574,1905-12-25 +1905-05-05 12:34:56.1575,1905-12-25 +1905-05-05 12:34:56.1576,1905-12-25 +1905-05-05 12:34:56.1577,1905-12-25 +1905-05-05 12:34:56.1578,1905-12-25 +1905-05-05 12:34:56.1579,1905-12-25 +1905-05-05 12:34:56.158,1905-12-25 +1905-05-05 12:34:56.1581,1905-12-25 +1905-05-05 12:34:56.1582,1905-12-25 +1905-05-05 12:34:56.1583,1905-12-25 +1905-05-05 12:34:56.1584,1905-12-25 +1905-05-05 12:34:56.1585,1905-12-25 +1905-05-05 12:34:56.1586,1905-12-25 +1905-05-05 12:34:56.1587,1905-12-25 +1905-05-05 12:34:56.1588,1905-12-25 +1905-05-05 12:34:56.1589,1905-12-25 +1905-05-05 12:34:56.159,1905-12-25 +1905-05-05 12:34:56.1591,1905-12-25 +1905-05-05 12:34:56.1592,1905-12-25 +1905-05-05 12:34:56.1593,1905-12-25 +1905-05-05 12:34:56.1594,1905-12-25 +1905-05-05 12:34:56.1595,1905-12-25 +1905-05-05 12:34:56.1596,1905-12-25 +1905-05-05 12:34:56.1597,1905-12-25 +1905-05-05 12:34:56.1598,1905-12-25 +1905-05-05 12:34:56.1599,1905-12-25 +1905-05-05 12:34:56.16,1905-12-25 +1905-05-05 12:34:56.1601,1905-12-25 +1905-05-05 12:34:56.1602,1905-12-25 +1905-05-05 12:34:56.1603,1905-12-25 +1905-05-05 12:34:56.1604,1905-12-25 +1905-05-05 12:34:56.1605,1905-12-25 +1905-05-05 12:34:56.1606,1905-12-25 +1905-05-05 12:34:56.1607,1905-12-25 +1905-05-05 12:34:56.1608,1905-12-25 +1905-05-05 12:34:56.1609,1905-12-25 +1905-05-05 12:34:56.161,1905-12-25 +1905-05-05 12:34:56.1611,1905-12-25 +1905-05-05 12:34:56.1612,1905-12-25 +1905-05-05 12:34:56.1613,1905-12-25 +1905-05-05 12:34:56.1614,1905-12-25 +1905-05-05 12:34:56.1615,1905-12-25 +1905-05-05 12:34:56.1616,1905-12-25 +1905-05-05 12:34:56.1617,1905-12-25 +1905-05-05 12:34:56.1618,1905-12-25 +1905-05-05 12:34:56.1619,1905-12-25 +1905-05-05 12:34:56.162,1905-12-25 +1905-05-05 12:34:56.1621,1905-12-25 +1905-05-05 12:34:56.1622,1905-12-25 +1905-05-05 12:34:56.1623,1905-12-25 +1905-05-05 12:34:56.1624,1905-12-25 +1905-05-05 12:34:56.1625,1905-12-25 +1905-05-05 12:34:56.1626,1905-12-25 +1905-05-05 12:34:56.1627,1905-12-25 +1905-05-05 12:34:56.1628,1905-12-25 +1905-05-05 12:34:56.1629,1905-12-25 +1905-05-05 12:34:56.163,1905-12-25 +1905-05-05 12:34:56.1631,1905-12-25 +1905-05-05 12:34:56.1632,1905-12-25 +1905-05-05 12:34:56.1633,1905-12-25 +1905-05-05 12:34:56.1634,1905-12-25 +1905-05-05 12:34:56.1635,1905-12-25 +1905-05-05 12:34:56.1636,1905-12-25 +1905-05-05 12:34:56.1637,1905-12-25 +1905-05-05 12:34:56.1638,1905-12-25 +1905-05-05 12:34:56.1639,1905-12-25 +1905-05-05 12:34:56.164,1905-12-25 +1905-05-05 12:34:56.1641,1905-12-25 +1905-05-05 12:34:56.1642,1905-12-25 +1905-05-05 12:34:56.1643,1905-12-25 +1905-05-05 12:34:56.1644,1905-12-25 +1905-05-05 12:34:56.1645,1905-12-25 +1905-05-05 12:34:56.1646,1905-12-25 +1905-05-05 12:34:56.1647,1905-12-25 +1905-05-05 12:34:56.1648,1905-12-25 +1905-05-05 12:34:56.1649,1905-12-25 +1905-05-05 12:34:56.165,1905-12-25 +1905-05-05 12:34:56.1651,1905-12-25 +1905-05-05 12:34:56.1652,1905-12-25 +1905-05-05 12:34:56.1653,1905-12-25 +1905-05-05 12:34:56.1654,1905-12-25 +1905-05-05 12:34:56.1655,1905-12-25 +1905-05-05 12:34:56.1656,1905-12-25 +1905-05-05 12:34:56.1657,1905-12-25 +1905-05-05 12:34:56.1658,1905-12-25 +1905-05-05 12:34:56.1659,1905-12-25 +1905-05-05 12:34:56.166,1905-12-25 +1905-05-05 12:34:56.1661,1905-12-25 +1905-05-05 12:34:56.1662,1905-12-25 +1905-05-05 12:34:56.1663,1905-12-25 +1905-05-05 12:34:56.1664,1905-12-25 +1905-05-05 12:34:56.1665,1905-12-25 +1905-05-05 12:34:56.1666,1905-12-25 +1905-05-05 12:34:56.1667,1905-12-25 +1905-05-05 12:34:56.1668,1905-12-25 +1905-05-05 12:34:56.1669,1905-12-25 +1905-05-05 12:34:56.167,1905-12-25 +1905-05-05 12:34:56.1671,1905-12-25 +1905-05-05 12:34:56.1672,1905-12-25 +1905-05-05 12:34:56.1673,1905-12-25 +1905-05-05 12:34:56.1674,1905-12-25 +1905-05-05 12:34:56.1675,1905-12-25 +1905-05-05 12:34:56.1676,1905-12-25 +1905-05-05 12:34:56.1677,1905-12-25 +1905-05-05 12:34:56.1678,1905-12-25 +1905-05-05 12:34:56.1679,1905-12-25 +1905-05-05 12:34:56.168,1905-12-25 +1905-05-05 12:34:56.1681,1905-12-25 +1905-05-05 12:34:56.1682,1905-12-25 +1905-05-05 12:34:56.1683,1905-12-25 +1905-05-05 12:34:56.1684,1905-12-25 +1905-05-05 12:34:56.1685,1905-12-25 +1905-05-05 12:34:56.1686,1905-12-25 +1905-05-05 12:34:56.1687,1905-12-25 +1905-05-05 12:34:56.1688,1905-12-25 +1905-05-05 12:34:56.1689,1905-12-25 +1905-05-05 12:34:56.169,1905-12-25 +1905-05-05 12:34:56.1691,1905-12-25 +1905-05-05 12:34:56.1692,1905-12-25 +1905-05-05 12:34:56.1693,1905-12-25 +1905-05-05 12:34:56.1694,1905-12-25 +1905-05-05 12:34:56.1695,1905-12-25 +1905-05-05 12:34:56.1696,1905-12-25 +1905-05-05 12:34:56.1697,1905-12-25 +1905-05-05 12:34:56.1698,1905-12-25 +1905-05-05 12:34:56.1699,1905-12-25 +1905-05-05 12:34:56.17,1905-12-25 +1905-05-05 12:34:56.1701,1905-12-25 +1905-05-05 12:34:56.1702,1905-12-25 +1905-05-05 12:34:56.1703,1905-12-25 +1905-05-05 12:34:56.1704,1905-12-25 +1905-05-05 12:34:56.1705,1905-12-25 +1905-05-05 12:34:56.1706,1905-12-25 +1905-05-05 12:34:56.1707,1905-12-25 +1905-05-05 12:34:56.1708,1905-12-25 +1905-05-05 12:34:56.1709,1905-12-25 +1905-05-05 12:34:56.171,1905-12-25 +1905-05-05 12:34:56.1711,1905-12-25 +1905-05-05 12:34:56.1712,1905-12-25 +1905-05-05 12:34:56.1713,1905-12-25 +1905-05-05 12:34:56.1714,1905-12-25 +1905-05-05 12:34:56.1715,1905-12-25 +1905-05-05 12:34:56.1716,1905-12-25 +1905-05-05 12:34:56.1717,1905-12-25 +1905-05-05 12:34:56.1718,1905-12-25 +1905-05-05 12:34:56.1719,1905-12-25 +1905-05-05 12:34:56.172,1905-12-25 +1905-05-05 12:34:56.1721,1905-12-25 +1905-05-05 12:34:56.1722,1905-12-25 +1905-05-05 12:34:56.1723,1905-12-25 +1905-05-05 12:34:56.1724,1905-12-25 +1905-05-05 12:34:56.1725,1905-12-25 +1905-05-05 12:34:56.1726,1905-12-25 +1905-05-05 12:34:56.1727,1905-12-25 +1905-05-05 12:34:56.1728,1905-12-25 +1905-05-05 12:34:56.1729,1905-12-25 +1905-05-05 12:34:56.173,1905-12-25 +1905-05-05 12:34:56.1731,1905-12-25 +1905-05-05 12:34:56.1732,1905-12-25 +1905-05-05 12:34:56.1733,1905-12-25 +1905-05-05 12:34:56.1734,1905-12-25 +1905-05-05 12:34:56.1735,1905-12-25 +1905-05-05 12:34:56.1736,1905-12-25 +1905-05-05 12:34:56.1737,1905-12-25 +1905-05-05 12:34:56.1738,1905-12-25 +1905-05-05 12:34:56.1739,1905-12-25 +1905-05-05 12:34:56.174,1905-12-25 +1905-05-05 12:34:56.1741,1905-12-25 +1905-05-05 12:34:56.1742,1905-12-25 +1905-05-05 12:34:56.1743,1905-12-25 +1905-05-05 12:34:56.1744,1905-12-25 +1905-05-05 12:34:56.1745,1905-12-25 +1905-05-05 12:34:56.1746,1905-12-25 +1905-05-05 12:34:56.1747,1905-12-25 +1905-05-05 12:34:56.1748,1905-12-25 +1905-05-05 12:34:56.1749,1905-12-25 +1905-05-05 12:34:56.175,1905-12-25 +1905-05-05 12:34:56.1751,1905-12-25 +1905-05-05 12:34:56.1752,1905-12-25 +1905-05-05 12:34:56.1753,1905-12-25 +1905-05-05 12:34:56.1754,1905-12-25 +1905-05-05 12:34:56.1755,1905-12-25 +1905-05-05 12:34:56.1756,1905-12-25 +1905-05-05 12:34:56.1757,1905-12-25 +1905-05-05 12:34:56.1758,1905-12-25 +1905-05-05 12:34:56.1759,1905-12-25 +1905-05-05 12:34:56.176,1905-12-25 +1905-05-05 12:34:56.1761,1905-12-25 +1905-05-05 12:34:56.1762,1905-12-25 +1905-05-05 12:34:56.1763,1905-12-25 +1905-05-05 12:34:56.1764,1905-12-25 +1905-05-05 12:34:56.1765,1905-12-25 +1905-05-05 12:34:56.1766,1905-12-25 +1905-05-05 12:34:56.1767,1905-12-25 +1905-05-05 12:34:56.1768,1905-12-25 +1905-05-05 12:34:56.1769,1905-12-25 +1905-05-05 12:34:56.177,1905-12-25 +1905-05-05 12:34:56.1771,1905-12-25 +1905-05-05 12:34:56.1772,1905-12-25 +1905-05-05 12:34:56.1773,1905-12-25 +1905-05-05 12:34:56.1774,1905-12-25 +1905-05-05 12:34:56.1775,1905-12-25 +1905-05-05 12:34:56.1776,1905-12-25 +1905-05-05 12:34:56.1777,1905-12-25 +1905-05-05 12:34:56.1778,1905-12-25 +1905-05-05 12:34:56.1779,1905-12-25 +1905-05-05 12:34:56.178,1905-12-25 +1905-05-05 12:34:56.1781,1905-12-25 +1905-05-05 12:34:56.1782,1905-12-25 +1905-05-05 12:34:56.1783,1905-12-25 +1905-05-05 12:34:56.1784,1905-12-25 +1905-05-05 12:34:56.1785,1905-12-25 +1905-05-05 12:34:56.1786,1905-12-25 +1905-05-05 12:34:56.1787,1905-12-25 +1905-05-05 12:34:56.1788,1905-12-25 +1905-05-05 12:34:56.1789,1905-12-25 +1905-05-05 12:34:56.179,1905-12-25 +1905-05-05 12:34:56.1791,1905-12-25 +1905-05-05 12:34:56.1792,1905-12-25 +1905-05-05 12:34:56.1793,1905-12-25 +1905-05-05 12:34:56.1794,1905-12-25 +1905-05-05 12:34:56.1795,1905-12-25 +1905-05-05 12:34:56.1796,1905-12-25 +1905-05-05 12:34:56.1797,1905-12-25 +1905-05-05 12:34:56.1798,1905-12-25 +1905-05-05 12:34:56.1799,1905-12-25 +1905-05-05 12:34:56.18,1905-12-25 +1905-05-05 12:34:56.1801,1905-12-25 +1905-05-05 12:34:56.1802,1905-12-25 +1905-05-05 12:34:56.1803,1905-12-25 +1905-05-05 12:34:56.1804,1905-12-25 +1905-05-05 12:34:56.1805,1905-12-25 +1905-05-05 12:34:56.1806,1905-12-25 +1905-05-05 12:34:56.1807,1905-12-25 +1905-05-05 12:34:56.1808,1905-12-25 +1905-05-05 12:34:56.1809,1905-12-25 +1905-05-05 12:34:56.181,1905-12-25 +1905-05-05 12:34:56.1811,1905-12-25 +1905-05-05 12:34:56.1812,1905-12-25 +1905-05-05 12:34:56.1813,1905-12-25 +1905-05-05 12:34:56.1814,1905-12-25 +1905-05-05 12:34:56.1815,1905-12-25 +1905-05-05 12:34:56.1816,1905-12-25 +1905-05-05 12:34:56.1817,1905-12-25 +1905-05-05 12:34:56.1818,1905-12-25 +1905-05-05 12:34:56.1819,1905-12-25 +1905-05-05 12:34:56.182,1905-12-25 +1905-05-05 12:34:56.1821,1905-12-25 +1905-05-05 12:34:56.1822,1905-12-25 +1905-05-05 12:34:56.1823,1905-12-25 +1905-05-05 12:34:56.1824,1905-12-25 +1905-05-05 12:34:56.1825,1905-12-25 +1905-05-05 12:34:56.1826,1905-12-25 +1905-05-05 12:34:56.1827,1905-12-25 +1905-05-05 12:34:56.1828,1905-12-25 +1905-05-05 12:34:56.1829,1905-12-25 +1905-05-05 12:34:56.183,1905-12-25 +1905-05-05 12:34:56.1831,1905-12-25 +1905-05-05 12:34:56.1832,1905-12-25 +1905-05-05 12:34:56.1833,1905-12-25 +1905-05-05 12:34:56.1834,1905-12-25 +1905-05-05 12:34:56.1835,1905-12-25 +1905-05-05 12:34:56.1836,1905-12-25 +1905-05-05 12:34:56.1837,1905-12-25 +1905-05-05 12:34:56.1838,1905-12-25 +1905-05-05 12:34:56.1839,1905-12-25 +1905-05-05 12:34:56.184,1905-12-25 +1905-05-05 12:34:56.1841,1905-12-25 +1905-05-05 12:34:56.1842,1905-12-25 +1905-05-05 12:34:56.1843,1905-12-25 +1905-05-05 12:34:56.1844,1905-12-25 +1905-05-05 12:34:56.1845,1905-12-25 +1905-05-05 12:34:56.1846,1905-12-25 +1905-05-05 12:34:56.1847,1905-12-25 +1905-05-05 12:34:56.1848,1905-12-25 +1905-05-05 12:34:56.1849,1905-12-25 +1905-05-05 12:34:56.185,1905-12-25 +1905-05-05 12:34:56.1851,1905-12-25 +1905-05-05 12:34:56.1852,1905-12-25 +1905-05-05 12:34:56.1853,1905-12-25 +1905-05-05 12:34:56.1854,1905-12-25 +1905-05-05 12:34:56.1855,1905-12-25 +1905-05-05 12:34:56.1856,1905-12-25 +1905-05-05 12:34:56.1857,1905-12-25 +1905-05-05 12:34:56.1858,1905-12-25 +1905-05-05 12:34:56.1859,1905-12-25 +1905-05-05 12:34:56.186,1905-12-25 +1905-05-05 12:34:56.1861,1905-12-25 +1905-05-05 12:34:56.1862,1905-12-25 +1905-05-05 12:34:56.1863,1905-12-25 +1905-05-05 12:34:56.1864,1905-12-25 +1905-05-05 12:34:56.1865,1905-12-25 +1905-05-05 12:34:56.1866,1905-12-25 +1905-05-05 12:34:56.1867,1905-12-25 +1905-05-05 12:34:56.1868,1905-12-25 +1905-05-05 12:34:56.1869,1905-12-25 +1905-05-05 12:34:56.187,1905-12-25 +1905-05-05 12:34:56.1871,1905-12-25 +1905-05-05 12:34:56.1872,1905-12-25 +1905-05-05 12:34:56.1873,1905-12-25 +1905-05-05 12:34:56.1874,1905-12-25 +1905-05-05 12:34:56.1875,1905-12-25 +1905-05-05 12:34:56.1876,1905-12-25 +1905-05-05 12:34:56.1877,1905-12-25 +1905-05-05 12:34:56.1878,1905-12-25 +1905-05-05 12:34:56.1879,1905-12-25 +1905-05-05 12:34:56.188,1905-12-25 +1905-05-05 12:34:56.1881,1905-12-25 +1905-05-05 12:34:56.1882,1905-12-25 +1905-05-05 12:34:56.1883,1905-12-25 +1905-05-05 12:34:56.1884,1905-12-25 +1905-05-05 12:34:56.1885,1905-12-25 +1905-05-05 12:34:56.1886,1905-12-25 +1905-05-05 12:34:56.1887,1905-12-25 +1905-05-05 12:34:56.1888,1905-12-25 +1905-05-05 12:34:56.1889,1905-12-25 +1905-05-05 12:34:56.189,1905-12-25 +1905-05-05 12:34:56.1891,1905-12-25 +1905-05-05 12:34:56.1892,1905-12-25 +1905-05-05 12:34:56.1893,1905-12-25 +1905-05-05 12:34:56.1894,1905-12-25 +1905-05-05 12:34:56.1895,1905-12-25 +1905-05-05 12:34:56.1896,1905-12-25 +1905-05-05 12:34:56.1897,1905-12-25 +1905-05-05 12:34:56.1898,1905-12-25 +1905-05-05 12:34:56.1899,1905-12-25 +1905-05-05 12:34:56.19,1905-12-25 +1905-05-05 12:34:56.1901,1905-12-25 +1905-05-05 12:34:56.1902,1905-12-25 +1905-05-05 12:34:56.1903,1905-12-25 +1905-05-05 12:34:56.1904,1905-12-25 +1905-05-05 12:34:56.1905,1905-12-25 +1905-05-05 12:34:56.1906,1905-12-25 +1905-05-05 12:34:56.1907,1905-12-25 +1905-05-05 12:34:56.1908,1905-12-25 +1905-05-05 12:34:56.1909,1905-12-25 +1905-05-05 12:34:56.191,1905-12-25 +1905-05-05 12:34:56.1911,1905-12-25 +1905-05-05 12:34:56.1912,1905-12-25 +1905-05-05 12:34:56.1913,1905-12-25 +1905-05-05 12:34:56.1914,1905-12-25 +1905-05-05 12:34:56.1915,1905-12-25 +1905-05-05 12:34:56.1916,1905-12-25 +1905-05-05 12:34:56.1917,1905-12-25 +1905-05-05 12:34:56.1918,1905-12-25 +1905-05-05 12:34:56.1919,1905-12-25 +1905-05-05 12:34:56.192,1905-12-25 +1905-05-05 12:34:56.1921,1905-12-25 +1905-05-05 12:34:56.1922,1905-12-25 +1905-05-05 12:34:56.1923,1905-12-25 +1905-05-05 12:34:56.1924,1905-12-25 +1905-05-05 12:34:56.1925,1905-12-25 +1905-05-05 12:34:56.1926,1905-12-25 +1905-05-05 12:34:56.1927,1905-12-25 +1905-05-05 12:34:56.1928,1905-12-25 +1905-05-05 12:34:56.1929,1905-12-25 +1905-05-05 12:34:56.193,1905-12-25 +1905-05-05 12:34:56.1931,1905-12-25 +1905-05-05 12:34:56.1932,1905-12-25 +1905-05-05 12:34:56.1933,1905-12-25 +1905-05-05 12:34:56.1934,1905-12-25 +1905-05-05 12:34:56.1935,1905-12-25 +1905-05-05 12:34:56.1936,1905-12-25 +1905-05-05 12:34:56.1937,1905-12-25 +1905-05-05 12:34:56.1938,1905-12-25 +1905-05-05 12:34:56.1939,1905-12-25 +1905-05-05 12:34:56.194,1905-12-25 +1905-05-05 12:34:56.1941,1905-12-25 +1905-05-05 12:34:56.1942,1905-12-25 +1905-05-05 12:34:56.1943,1905-12-25 +1905-05-05 12:34:56.1944,1905-12-25 +1905-05-05 12:34:56.1945,1905-12-25 +1905-05-05 12:34:56.1946,1905-12-25 +1905-05-05 12:34:56.1947,1905-12-25 +1905-05-05 12:34:56.1948,1905-12-25 +1905-05-05 12:34:56.1949,1905-12-25 +1905-05-05 12:34:56.195,1905-12-25 +1905-05-05 12:34:56.1951,1905-12-25 +1905-05-05 12:34:56.1952,1905-12-25 +1905-05-05 12:34:56.1953,1905-12-25 +1905-05-05 12:34:56.1954,1905-12-25 +1905-05-05 12:34:56.1955,1905-12-25 +1905-05-05 12:34:56.1956,1905-12-25 +1905-05-05 12:34:56.1957,1905-12-25 +1905-05-05 12:34:56.1958,1905-12-25 +1905-05-05 12:34:56.1959,1905-12-25 +1905-05-05 12:34:56.196,1905-12-25 +1905-05-05 12:34:56.1961,1905-12-25 +1905-05-05 12:34:56.1962,1905-12-25 +1905-05-05 12:34:56.1963,1905-12-25 +1905-05-05 12:34:56.1964,1905-12-25 +1905-05-05 12:34:56.1965,1905-12-25 +1905-05-05 12:34:56.1966,1905-12-25 +1905-05-05 12:34:56.1967,1905-12-25 +1905-05-05 12:34:56.1968,1905-12-25 +1905-05-05 12:34:56.1969,1905-12-25 +1905-05-05 12:34:56.197,1905-12-25 +1905-05-05 12:34:56.1971,1905-12-25 +1905-05-05 12:34:56.1972,1905-12-25 +1905-05-05 12:34:56.1973,1905-12-25 +1905-05-05 12:34:56.1974,1905-12-25 +1905-05-05 12:34:56.1975,1905-12-25 +1905-05-05 12:34:56.1976,1905-12-25 +1905-05-05 12:34:56.1977,1905-12-25 +1905-05-05 12:34:56.1978,1905-12-25 +1905-05-05 12:34:56.1979,1905-12-25 +1905-05-05 12:34:56.198,1905-12-25 +1905-05-05 12:34:56.1981,1905-12-25 +1905-05-05 12:34:56.1982,1905-12-25 +1905-05-05 12:34:56.1983,1905-12-25 +1905-05-05 12:34:56.1984,1905-12-25 +1905-05-05 12:34:56.1985,1905-12-25 +1905-05-05 12:34:56.1986,1905-12-25 +1905-05-05 12:34:56.1987,1905-12-25 +1905-05-05 12:34:56.1988,1905-12-25 +1905-05-05 12:34:56.1989,1905-12-25 +1905-05-05 12:34:56.199,1905-12-25 +1905-05-05 12:34:56.1991,1905-12-25 +1905-05-05 12:34:56.1992,1905-12-25 +1905-05-05 12:34:56.1993,1905-12-25 +1905-05-05 12:34:56.1994,1905-12-25 +1905-05-05 12:34:56.1995,1905-12-25 +1905-05-05 12:34:56.1996,1905-12-25 +1905-05-05 12:34:56.1997,1905-12-25 +1905-05-05 12:34:56.1998,1905-12-25 +1905-05-05 12:34:56.1999,1905-12-25 +1906-05-05 12:34:56.1,1906-12-25 +1906-05-05 12:34:56.1001,1906-12-25 +1906-05-05 12:34:56.1002,1906-12-25 +1906-05-05 12:34:56.1003,1906-12-25 +1906-05-05 12:34:56.1004,1906-12-25 +1906-05-05 12:34:56.1005,1906-12-25 +1906-05-05 12:34:56.1006,1906-12-25 +1906-05-05 12:34:56.1007,1906-12-25 +1906-05-05 12:34:56.1008,1906-12-25 +1906-05-05 12:34:56.1009,1906-12-25 +1906-05-05 12:34:56.101,1906-12-25 +1906-05-05 12:34:56.1011,1906-12-25 +1906-05-05 12:34:56.1012,1906-12-25 +1906-05-05 12:34:56.1013,1906-12-25 +1906-05-05 12:34:56.1014,1906-12-25 +1906-05-05 12:34:56.1015,1906-12-25 +1906-05-05 12:34:56.1016,1906-12-25 +1906-05-05 12:34:56.1017,1906-12-25 +1906-05-05 12:34:56.1018,1906-12-25 +1906-05-05 12:34:56.1019,1906-12-25 +1906-05-05 12:34:56.102,1906-12-25 +1906-05-05 12:34:56.1021,1906-12-25 +1906-05-05 12:34:56.1022,1906-12-25 +1906-05-05 12:34:56.1023,1906-12-25 +1906-05-05 12:34:56.1024,1906-12-25 +1906-05-05 12:34:56.1025,1906-12-25 +1906-05-05 12:34:56.1026,1906-12-25 +1906-05-05 12:34:56.1027,1906-12-25 +1906-05-05 12:34:56.1028,1906-12-25 +1906-05-05 12:34:56.1029,1906-12-25 +1906-05-05 12:34:56.103,1906-12-25 +1906-05-05 12:34:56.1031,1906-12-25 +1906-05-05 12:34:56.1032,1906-12-25 +1906-05-05 12:34:56.1033,1906-12-25 +1906-05-05 12:34:56.1034,1906-12-25 +1906-05-05 12:34:56.1035,1906-12-25 +1906-05-05 12:34:56.1036,1906-12-25 +1906-05-05 12:34:56.1037,1906-12-25 +1906-05-05 12:34:56.1038,1906-12-25 +1906-05-05 12:34:56.1039,1906-12-25 +1906-05-05 12:34:56.104,1906-12-25 +1906-05-05 12:34:56.1041,1906-12-25 +1906-05-05 12:34:56.1042,1906-12-25 +1906-05-05 12:34:56.1043,1906-12-25 +1906-05-05 12:34:56.1044,1906-12-25 +1906-05-05 12:34:56.1045,1906-12-25 +1906-05-05 12:34:56.1046,1906-12-25 +1906-05-05 12:34:56.1047,1906-12-25 +1906-05-05 12:34:56.1048,1906-12-25 +1906-05-05 12:34:56.1049,1906-12-25 +1906-05-05 12:34:56.105,1906-12-25 +1906-05-05 12:34:56.1051,1906-12-25 +1906-05-05 12:34:56.1052,1906-12-25 +1906-05-05 12:34:56.1053,1906-12-25 +1906-05-05 12:34:56.1054,1906-12-25 +1906-05-05 12:34:56.1055,1906-12-25 +1906-05-05 12:34:56.1056,1906-12-25 +1906-05-05 12:34:56.1057,1906-12-25 +1906-05-05 12:34:56.1058,1906-12-25 +1906-05-05 12:34:56.1059,1906-12-25 +1906-05-05 12:34:56.106,1906-12-25 +1906-05-05 12:34:56.1061,1906-12-25 +1906-05-05 12:34:56.1062,1906-12-25 +1906-05-05 12:34:56.1063,1906-12-25 +1906-05-05 12:34:56.1064,1906-12-25 +1906-05-05 12:34:56.1065,1906-12-25 +1906-05-05 12:34:56.1066,1906-12-25 +1906-05-05 12:34:56.1067,1906-12-25 +1906-05-05 12:34:56.1068,1906-12-25 +1906-05-05 12:34:56.1069,1906-12-25 +1906-05-05 12:34:56.107,1906-12-25 +1906-05-05 12:34:56.1071,1906-12-25 +1906-05-05 12:34:56.1072,1906-12-25 +1906-05-05 12:34:56.1073,1906-12-25 +1906-05-05 12:34:56.1074,1906-12-25 +1906-05-05 12:34:56.1075,1906-12-25 +1906-05-05 12:34:56.1076,1906-12-25 +1906-05-05 12:34:56.1077,1906-12-25 +1906-05-05 12:34:56.1078,1906-12-25 +1906-05-05 12:34:56.1079,1906-12-25 +1906-05-05 12:34:56.108,1906-12-25 +1906-05-05 12:34:56.1081,1906-12-25 +1906-05-05 12:34:56.1082,1906-12-25 +1906-05-05 12:34:56.1083,1906-12-25 +1906-05-05 12:34:56.1084,1906-12-25 +1906-05-05 12:34:56.1085,1906-12-25 +1906-05-05 12:34:56.1086,1906-12-25 +1906-05-05 12:34:56.1087,1906-12-25 +1906-05-05 12:34:56.1088,1906-12-25 +1906-05-05 12:34:56.1089,1906-12-25 +1906-05-05 12:34:56.109,1906-12-25 +1906-05-05 12:34:56.1091,1906-12-25 +1906-05-05 12:34:56.1092,1906-12-25 +1906-05-05 12:34:56.1093,1906-12-25 +1906-05-05 12:34:56.1094,1906-12-25 +1906-05-05 12:34:56.1095,1906-12-25 +1906-05-05 12:34:56.1096,1906-12-25 +1906-05-05 12:34:56.1097,1906-12-25 +1906-05-05 12:34:56.1098,1906-12-25 +1906-05-05 12:34:56.1099,1906-12-25 +1906-05-05 12:34:56.11,1906-12-25 +1906-05-05 12:34:56.1101,1906-12-25 +1906-05-05 12:34:56.1102,1906-12-25 +1906-05-05 12:34:56.1103,1906-12-25 +1906-05-05 12:34:56.1104,1906-12-25 +1906-05-05 12:34:56.1105,1906-12-25 +1906-05-05 12:34:56.1106,1906-12-25 +1906-05-05 12:34:56.1107,1906-12-25 +1906-05-05 12:34:56.1108,1906-12-25 +1906-05-05 12:34:56.1109,1906-12-25 +1906-05-05 12:34:56.111,1906-12-25 +1906-05-05 12:34:56.1111,1906-12-25 +1906-05-05 12:34:56.1112,1906-12-25 +1906-05-05 12:34:56.1113,1906-12-25 +1906-05-05 12:34:56.1114,1906-12-25 +1906-05-05 12:34:56.1115,1906-12-25 +1906-05-05 12:34:56.1116,1906-12-25 +1906-05-05 12:34:56.1117,1906-12-25 +1906-05-05 12:34:56.1118,1906-12-25 +1906-05-05 12:34:56.1119,1906-12-25 +1906-05-05 12:34:56.112,1906-12-25 +1906-05-05 12:34:56.1121,1906-12-25 +1906-05-05 12:34:56.1122,1906-12-25 +1906-05-05 12:34:56.1123,1906-12-25 +1906-05-05 12:34:56.1124,1906-12-25 +1906-05-05 12:34:56.1125,1906-12-25 +1906-05-05 12:34:56.1126,1906-12-25 +1906-05-05 12:34:56.1127,1906-12-25 +1906-05-05 12:34:56.1128,1906-12-25 +1906-05-05 12:34:56.1129,1906-12-25 +1906-05-05 12:34:56.113,1906-12-25 +1906-05-05 12:34:56.1131,1906-12-25 +1906-05-05 12:34:56.1132,1906-12-25 +1906-05-05 12:34:56.1133,1906-12-25 +1906-05-05 12:34:56.1134,1906-12-25 +1906-05-05 12:34:56.1135,1906-12-25 +1906-05-05 12:34:56.1136,1906-12-25 +1906-05-05 12:34:56.1137,1906-12-25 +1906-05-05 12:34:56.1138,1906-12-25 +1906-05-05 12:34:56.1139,1906-12-25 +1906-05-05 12:34:56.114,1906-12-25 +1906-05-05 12:34:56.1141,1906-12-25 +1906-05-05 12:34:56.1142,1906-12-25 +1906-05-05 12:34:56.1143,1906-12-25 +1906-05-05 12:34:56.1144,1906-12-25 +1906-05-05 12:34:56.1145,1906-12-25 +1906-05-05 12:34:56.1146,1906-12-25 +1906-05-05 12:34:56.1147,1906-12-25 +1906-05-05 12:34:56.1148,1906-12-25 +1906-05-05 12:34:56.1149,1906-12-25 +1906-05-05 12:34:56.115,1906-12-25 +1906-05-05 12:34:56.1151,1906-12-25 +1906-05-05 12:34:56.1152,1906-12-25 +1906-05-05 12:34:56.1153,1906-12-25 +1906-05-05 12:34:56.1154,1906-12-25 +1906-05-05 12:34:56.1155,1906-12-25 +1906-05-05 12:34:56.1156,1906-12-25 +1906-05-05 12:34:56.1157,1906-12-25 +1906-05-05 12:34:56.1158,1906-12-25 +1906-05-05 12:34:56.1159,1906-12-25 +1906-05-05 12:34:56.116,1906-12-25 +1906-05-05 12:34:56.1161,1906-12-25 +1906-05-05 12:34:56.1162,1906-12-25 +1906-05-05 12:34:56.1163,1906-12-25 +1906-05-05 12:34:56.1164,1906-12-25 +1906-05-05 12:34:56.1165,1906-12-25 +1906-05-05 12:34:56.1166,1906-12-25 +1906-05-05 12:34:56.1167,1906-12-25 +1906-05-05 12:34:56.1168,1906-12-25 +1906-05-05 12:34:56.1169,1906-12-25 +1906-05-05 12:34:56.117,1906-12-25 +1906-05-05 12:34:56.1171,1906-12-25 +1906-05-05 12:34:56.1172,1906-12-25 +1906-05-05 12:34:56.1173,1906-12-25 +1906-05-05 12:34:56.1174,1906-12-25 +1906-05-05 12:34:56.1175,1906-12-25 +1906-05-05 12:34:56.1176,1906-12-25 +1906-05-05 12:34:56.1177,1906-12-25 +1906-05-05 12:34:56.1178,1906-12-25 +1906-05-05 12:34:56.1179,1906-12-25 +1906-05-05 12:34:56.118,1906-12-25 +1906-05-05 12:34:56.1181,1906-12-25 +1906-05-05 12:34:56.1182,1906-12-25 +1906-05-05 12:34:56.1183,1906-12-25 +1906-05-05 12:34:56.1184,1906-12-25 +1906-05-05 12:34:56.1185,1906-12-25 +1906-05-05 12:34:56.1186,1906-12-25 +1906-05-05 12:34:56.1187,1906-12-25 +1906-05-05 12:34:56.1188,1906-12-25 +1906-05-05 12:34:56.1189,1906-12-25 +1906-05-05 12:34:56.119,1906-12-25 +1906-05-05 12:34:56.1191,1906-12-25 +1906-05-05 12:34:56.1192,1906-12-25 +1906-05-05 12:34:56.1193,1906-12-25 +1906-05-05 12:34:56.1194,1906-12-25 +1906-05-05 12:34:56.1195,1906-12-25 +1906-05-05 12:34:56.1196,1906-12-25 +1906-05-05 12:34:56.1197,1906-12-25 +1906-05-05 12:34:56.1198,1906-12-25 +1906-05-05 12:34:56.1199,1906-12-25 +1906-05-05 12:34:56.12,1906-12-25 +1906-05-05 12:34:56.1201,1906-12-25 +1906-05-05 12:34:56.1202,1906-12-25 +1906-05-05 12:34:56.1203,1906-12-25 +1906-05-05 12:34:56.1204,1906-12-25 +1906-05-05 12:34:56.1205,1906-12-25 +1906-05-05 12:34:56.1206,1906-12-25 +1906-05-05 12:34:56.1207,1906-12-25 +1906-05-05 12:34:56.1208,1906-12-25 +1906-05-05 12:34:56.1209,1906-12-25 +1906-05-05 12:34:56.121,1906-12-25 +1906-05-05 12:34:56.1211,1906-12-25 +1906-05-05 12:34:56.1212,1906-12-25 +1906-05-05 12:34:56.1213,1906-12-25 +1906-05-05 12:34:56.1214,1906-12-25 +1906-05-05 12:34:56.1215,1906-12-25 +1906-05-05 12:34:56.1216,1906-12-25 +1906-05-05 12:34:56.1217,1906-12-25 +1906-05-05 12:34:56.1218,1906-12-25 +1906-05-05 12:34:56.1219,1906-12-25 +1906-05-05 12:34:56.122,1906-12-25 +1906-05-05 12:34:56.1221,1906-12-25 +1906-05-05 12:34:56.1222,1906-12-25 +1906-05-05 12:34:56.1223,1906-12-25 +1906-05-05 12:34:56.1224,1906-12-25 +1906-05-05 12:34:56.1225,1906-12-25 +1906-05-05 12:34:56.1226,1906-12-25 +1906-05-05 12:34:56.1227,1906-12-25 +1906-05-05 12:34:56.1228,1906-12-25 +1906-05-05 12:34:56.1229,1906-12-25 +1906-05-05 12:34:56.123,1906-12-25 +1906-05-05 12:34:56.1231,1906-12-25 +1906-05-05 12:34:56.1232,1906-12-25 +1906-05-05 12:34:56.1233,1906-12-25 +1906-05-05 12:34:56.1234,1906-12-25 +1906-05-05 12:34:56.1235,1906-12-25 +1906-05-05 12:34:56.1236,1906-12-25 +1906-05-05 12:34:56.1237,1906-12-25 +1906-05-05 12:34:56.1238,1906-12-25 +1906-05-05 12:34:56.1239,1906-12-25 +1906-05-05 12:34:56.124,1906-12-25 +1906-05-05 12:34:56.1241,1906-12-25 +1906-05-05 12:34:56.1242,1906-12-25 +1906-05-05 12:34:56.1243,1906-12-25 +1906-05-05 12:34:56.1244,1906-12-25 +1906-05-05 12:34:56.1245,1906-12-25 +1906-05-05 12:34:56.1246,1906-12-25 +1906-05-05 12:34:56.1247,1906-12-25 +1906-05-05 12:34:56.1248,1906-12-25 +1906-05-05 12:34:56.1249,1906-12-25 +1906-05-05 12:34:56.125,1906-12-25 +1906-05-05 12:34:56.1251,1906-12-25 +1906-05-05 12:34:56.1252,1906-12-25 +1906-05-05 12:34:56.1253,1906-12-25 +1906-05-05 12:34:56.1254,1906-12-25 +1906-05-05 12:34:56.1255,1906-12-25 +1906-05-05 12:34:56.1256,1906-12-25 +1906-05-05 12:34:56.1257,1906-12-25 +1906-05-05 12:34:56.1258,1906-12-25 +1906-05-05 12:34:56.1259,1906-12-25 +1906-05-05 12:34:56.126,1906-12-25 +1906-05-05 12:34:56.1261,1906-12-25 +1906-05-05 12:34:56.1262,1906-12-25 +1906-05-05 12:34:56.1263,1906-12-25 +1906-05-05 12:34:56.1264,1906-12-25 +1906-05-05 12:34:56.1265,1906-12-25 +1906-05-05 12:34:56.1266,1906-12-25 +1906-05-05 12:34:56.1267,1906-12-25 +1906-05-05 12:34:56.1268,1906-12-25 +1906-05-05 12:34:56.1269,1906-12-25 +1906-05-05 12:34:56.127,1906-12-25 +1906-05-05 12:34:56.1271,1906-12-25 +1906-05-05 12:34:56.1272,1906-12-25 +1906-05-05 12:34:56.1273,1906-12-25 +1906-05-05 12:34:56.1274,1906-12-25 +1906-05-05 12:34:56.1275,1906-12-25 +1906-05-05 12:34:56.1276,1906-12-25 +1906-05-05 12:34:56.1277,1906-12-25 +1906-05-05 12:34:56.1278,1906-12-25 +1906-05-05 12:34:56.1279,1906-12-25 +1906-05-05 12:34:56.128,1906-12-25 +1906-05-05 12:34:56.1281,1906-12-25 +1906-05-05 12:34:56.1282,1906-12-25 +1906-05-05 12:34:56.1283,1906-12-25 +1906-05-05 12:34:56.1284,1906-12-25 +1906-05-05 12:34:56.1285,1906-12-25 +1906-05-05 12:34:56.1286,1906-12-25 +1906-05-05 12:34:56.1287,1906-12-25 +1906-05-05 12:34:56.1288,1906-12-25 +1906-05-05 12:34:56.1289,1906-12-25 +1906-05-05 12:34:56.129,1906-12-25 +1906-05-05 12:34:56.1291,1906-12-25 +1906-05-05 12:34:56.1292,1906-12-25 +1906-05-05 12:34:56.1293,1906-12-25 +1906-05-05 12:34:56.1294,1906-12-25 +1906-05-05 12:34:56.1295,1906-12-25 +1906-05-05 12:34:56.1296,1906-12-25 +1906-05-05 12:34:56.1297,1906-12-25 +1906-05-05 12:34:56.1298,1906-12-25 +1906-05-05 12:34:56.1299,1906-12-25 +1906-05-05 12:34:56.13,1906-12-25 +1906-05-05 12:34:56.1301,1906-12-25 +1906-05-05 12:34:56.1302,1906-12-25 +1906-05-05 12:34:56.1303,1906-12-25 +1906-05-05 12:34:56.1304,1906-12-25 +1906-05-05 12:34:56.1305,1906-12-25 +1906-05-05 12:34:56.1306,1906-12-25 +1906-05-05 12:34:56.1307,1906-12-25 +1906-05-05 12:34:56.1308,1906-12-25 +1906-05-05 12:34:56.1309,1906-12-25 +1906-05-05 12:34:56.131,1906-12-25 +1906-05-05 12:34:56.1311,1906-12-25 +1906-05-05 12:34:56.1312,1906-12-25 +1906-05-05 12:34:56.1313,1906-12-25 +1906-05-05 12:34:56.1314,1906-12-25 +1906-05-05 12:34:56.1315,1906-12-25 +1906-05-05 12:34:56.1316,1906-12-25 +1906-05-05 12:34:56.1317,1906-12-25 +1906-05-05 12:34:56.1318,1906-12-25 +1906-05-05 12:34:56.1319,1906-12-25 +1906-05-05 12:34:56.132,1906-12-25 +1906-05-05 12:34:56.1321,1906-12-25 +1906-05-05 12:34:56.1322,1906-12-25 +1906-05-05 12:34:56.1323,1906-12-25 +1906-05-05 12:34:56.1324,1906-12-25 +1906-05-05 12:34:56.1325,1906-12-25 +1906-05-05 12:34:56.1326,1906-12-25 +1906-05-05 12:34:56.1327,1906-12-25 +1906-05-05 12:34:56.1328,1906-12-25 +1906-05-05 12:34:56.1329,1906-12-25 +1906-05-05 12:34:56.133,1906-12-25 +1906-05-05 12:34:56.1331,1906-12-25 +1906-05-05 12:34:56.1332,1906-12-25 +1906-05-05 12:34:56.1333,1906-12-25 +1906-05-05 12:34:56.1334,1906-12-25 +1906-05-05 12:34:56.1335,1906-12-25 +1906-05-05 12:34:56.1336,1906-12-25 +1906-05-05 12:34:56.1337,1906-12-25 +1906-05-05 12:34:56.1338,1906-12-25 +1906-05-05 12:34:56.1339,1906-12-25 +1906-05-05 12:34:56.134,1906-12-25 +1906-05-05 12:34:56.1341,1906-12-25 +1906-05-05 12:34:56.1342,1906-12-25 +1906-05-05 12:34:56.1343,1906-12-25 +1906-05-05 12:34:56.1344,1906-12-25 +1906-05-05 12:34:56.1345,1906-12-25 +1906-05-05 12:34:56.1346,1906-12-25 +1906-05-05 12:34:56.1347,1906-12-25 +1906-05-05 12:34:56.1348,1906-12-25 +1906-05-05 12:34:56.1349,1906-12-25 +1906-05-05 12:34:56.135,1906-12-25 +1906-05-05 12:34:56.1351,1906-12-25 +1906-05-05 12:34:56.1352,1906-12-25 +1906-05-05 12:34:56.1353,1906-12-25 +1906-05-05 12:34:56.1354,1906-12-25 +1906-05-05 12:34:56.1355,1906-12-25 +1906-05-05 12:34:56.1356,1906-12-25 +1906-05-05 12:34:56.1357,1906-12-25 +1906-05-05 12:34:56.1358,1906-12-25 +1906-05-05 12:34:56.1359,1906-12-25 +1906-05-05 12:34:56.136,1906-12-25 +1906-05-05 12:34:56.1361,1906-12-25 +1906-05-05 12:34:56.1362,1906-12-25 +1906-05-05 12:34:56.1363,1906-12-25 +1906-05-05 12:34:56.1364,1906-12-25 +1906-05-05 12:34:56.1365,1906-12-25 +1906-05-05 12:34:56.1366,1906-12-25 +1906-05-05 12:34:56.1367,1906-12-25 +1906-05-05 12:34:56.1368,1906-12-25 +1906-05-05 12:34:56.1369,1906-12-25 +1906-05-05 12:34:56.137,1906-12-25 +1906-05-05 12:34:56.1371,1906-12-25 +1906-05-05 12:34:56.1372,1906-12-25 +1906-05-05 12:34:56.1373,1906-12-25 +1906-05-05 12:34:56.1374,1906-12-25 +1906-05-05 12:34:56.1375,1906-12-25 +1906-05-05 12:34:56.1376,1906-12-25 +1906-05-05 12:34:56.1377,1906-12-25 +1906-05-05 12:34:56.1378,1906-12-25 +1906-05-05 12:34:56.1379,1906-12-25 +1906-05-05 12:34:56.138,1906-12-25 +1906-05-05 12:34:56.1381,1906-12-25 +1906-05-05 12:34:56.1382,1906-12-25 +1906-05-05 12:34:56.1383,1906-12-25 +1906-05-05 12:34:56.1384,1906-12-25 +1906-05-05 12:34:56.1385,1906-12-25 +1906-05-05 12:34:56.1386,1906-12-25 +1906-05-05 12:34:56.1387,1906-12-25 +1906-05-05 12:34:56.1388,1906-12-25 +1906-05-05 12:34:56.1389,1906-12-25 +1906-05-05 12:34:56.139,1906-12-25 +1906-05-05 12:34:56.1391,1906-12-25 +1906-05-05 12:34:56.1392,1906-12-25 +1906-05-05 12:34:56.1393,1906-12-25 +1906-05-05 12:34:56.1394,1906-12-25 +1906-05-05 12:34:56.1395,1906-12-25 +1906-05-05 12:34:56.1396,1906-12-25 +1906-05-05 12:34:56.1397,1906-12-25 +1906-05-05 12:34:56.1398,1906-12-25 +1906-05-05 12:34:56.1399,1906-12-25 +1906-05-05 12:34:56.14,1906-12-25 +1906-05-05 12:34:56.1401,1906-12-25 +1906-05-05 12:34:56.1402,1906-12-25 +1906-05-05 12:34:56.1403,1906-12-25 +1906-05-05 12:34:56.1404,1906-12-25 +1906-05-05 12:34:56.1405,1906-12-25 +1906-05-05 12:34:56.1406,1906-12-25 +1906-05-05 12:34:56.1407,1906-12-25 +1906-05-05 12:34:56.1408,1906-12-25 +1906-05-05 12:34:56.1409,1906-12-25 +1906-05-05 12:34:56.141,1906-12-25 +1906-05-05 12:34:56.1411,1906-12-25 +1906-05-05 12:34:56.1412,1906-12-25 +1906-05-05 12:34:56.1413,1906-12-25 +1906-05-05 12:34:56.1414,1906-12-25 +1906-05-05 12:34:56.1415,1906-12-25 +1906-05-05 12:34:56.1416,1906-12-25 +1906-05-05 12:34:56.1417,1906-12-25 +1906-05-05 12:34:56.1418,1906-12-25 +1906-05-05 12:34:56.1419,1906-12-25 +1906-05-05 12:34:56.142,1906-12-25 +1906-05-05 12:34:56.1421,1906-12-25 +1906-05-05 12:34:56.1422,1906-12-25 +1906-05-05 12:34:56.1423,1906-12-25 +1906-05-05 12:34:56.1424,1906-12-25 +1906-05-05 12:34:56.1425,1906-12-25 +1906-05-05 12:34:56.1426,1906-12-25 +1906-05-05 12:34:56.1427,1906-12-25 +1906-05-05 12:34:56.1428,1906-12-25 +1906-05-05 12:34:56.1429,1906-12-25 +1906-05-05 12:34:56.143,1906-12-25 +1906-05-05 12:34:56.1431,1906-12-25 +1906-05-05 12:34:56.1432,1906-12-25 +1906-05-05 12:34:56.1433,1906-12-25 +1906-05-05 12:34:56.1434,1906-12-25 +1906-05-05 12:34:56.1435,1906-12-25 +1906-05-05 12:34:56.1436,1906-12-25 +1906-05-05 12:34:56.1437,1906-12-25 +1906-05-05 12:34:56.1438,1906-12-25 +1906-05-05 12:34:56.1439,1906-12-25 +1906-05-05 12:34:56.144,1906-12-25 +1906-05-05 12:34:56.1441,1906-12-25 +1906-05-05 12:34:56.1442,1906-12-25 +1906-05-05 12:34:56.1443,1906-12-25 +1906-05-05 12:34:56.1444,1906-12-25 +1906-05-05 12:34:56.1445,1906-12-25 +1906-05-05 12:34:56.1446,1906-12-25 +1906-05-05 12:34:56.1447,1906-12-25 +1906-05-05 12:34:56.1448,1906-12-25 +1906-05-05 12:34:56.1449,1906-12-25 +1906-05-05 12:34:56.145,1906-12-25 +1906-05-05 12:34:56.1451,1906-12-25 +1906-05-05 12:34:56.1452,1906-12-25 +1906-05-05 12:34:56.1453,1906-12-25 +1906-05-05 12:34:56.1454,1906-12-25 +1906-05-05 12:34:56.1455,1906-12-25 +1906-05-05 12:34:56.1456,1906-12-25 +1906-05-05 12:34:56.1457,1906-12-25 +1906-05-05 12:34:56.1458,1906-12-25 +1906-05-05 12:34:56.1459,1906-12-25 +1906-05-05 12:34:56.146,1906-12-25 +1906-05-05 12:34:56.1461,1906-12-25 +1906-05-05 12:34:56.1462,1906-12-25 +1906-05-05 12:34:56.1463,1906-12-25 +1906-05-05 12:34:56.1464,1906-12-25 +1906-05-05 12:34:56.1465,1906-12-25 +1906-05-05 12:34:56.1466,1906-12-25 +1906-05-05 12:34:56.1467,1906-12-25 +1906-05-05 12:34:56.1468,1906-12-25 +1906-05-05 12:34:56.1469,1906-12-25 +1906-05-05 12:34:56.147,1906-12-25 +1906-05-05 12:34:56.1471,1906-12-25 +1906-05-05 12:34:56.1472,1906-12-25 +1906-05-05 12:34:56.1473,1906-12-25 +1906-05-05 12:34:56.1474,1906-12-25 +1906-05-05 12:34:56.1475,1906-12-25 +1906-05-05 12:34:56.1476,1906-12-25 +1906-05-05 12:34:56.1477,1906-12-25 +1906-05-05 12:34:56.1478,1906-12-25 +1906-05-05 12:34:56.1479,1906-12-25 +1906-05-05 12:34:56.148,1906-12-25 +1906-05-05 12:34:56.1481,1906-12-25 +1906-05-05 12:34:56.1482,1906-12-25 +1906-05-05 12:34:56.1483,1906-12-25 +1906-05-05 12:34:56.1484,1906-12-25 +1906-05-05 12:34:56.1485,1906-12-25 +1906-05-05 12:34:56.1486,1906-12-25 +1906-05-05 12:34:56.1487,1906-12-25 +1906-05-05 12:34:56.1488,1906-12-25 +1906-05-05 12:34:56.1489,1906-12-25 +1906-05-05 12:34:56.149,1906-12-25 +1906-05-05 12:34:56.1491,1906-12-25 +1906-05-05 12:34:56.1492,1906-12-25 +1906-05-05 12:34:56.1493,1906-12-25 +1906-05-05 12:34:56.1494,1906-12-25 +1906-05-05 12:34:56.1495,1906-12-25 +1906-05-05 12:34:56.1496,1906-12-25 +1906-05-05 12:34:56.1497,1906-12-25 +1906-05-05 12:34:56.1498,1906-12-25 +1906-05-05 12:34:56.1499,1906-12-25 +1906-05-05 12:34:56.15,1906-12-25 +1906-05-05 12:34:56.1501,1906-12-25 +1906-05-05 12:34:56.1502,1906-12-25 +1906-05-05 12:34:56.1503,1906-12-25 +1906-05-05 12:34:56.1504,1906-12-25 +1906-05-05 12:34:56.1505,1906-12-25 +1906-05-05 12:34:56.1506,1906-12-25 +1906-05-05 12:34:56.1507,1906-12-25 +1906-05-05 12:34:56.1508,1906-12-25 +1906-05-05 12:34:56.1509,1906-12-25 +1906-05-05 12:34:56.151,1906-12-25 +1906-05-05 12:34:56.1511,1906-12-25 +1906-05-05 12:34:56.1512,1906-12-25 +1906-05-05 12:34:56.1513,1906-12-25 +1906-05-05 12:34:56.1514,1906-12-25 +1906-05-05 12:34:56.1515,1906-12-25 +1906-05-05 12:34:56.1516,1906-12-25 +1906-05-05 12:34:56.1517,1906-12-25 +1906-05-05 12:34:56.1518,1906-12-25 +1906-05-05 12:34:56.1519,1906-12-25 +1906-05-05 12:34:56.152,1906-12-25 +1906-05-05 12:34:56.1521,1906-12-25 +1906-05-05 12:34:56.1522,1906-12-25 +1906-05-05 12:34:56.1523,1906-12-25 +1906-05-05 12:34:56.1524,1906-12-25 +1906-05-05 12:34:56.1525,1906-12-25 +1906-05-05 12:34:56.1526,1906-12-25 +1906-05-05 12:34:56.1527,1906-12-25 +1906-05-05 12:34:56.1528,1906-12-25 +1906-05-05 12:34:56.1529,1906-12-25 +1906-05-05 12:34:56.153,1906-12-25 +1906-05-05 12:34:56.1531,1906-12-25 +1906-05-05 12:34:56.1532,1906-12-25 +1906-05-05 12:34:56.1533,1906-12-25 +1906-05-05 12:34:56.1534,1906-12-25 +1906-05-05 12:34:56.1535,1906-12-25 +1906-05-05 12:34:56.1536,1906-12-25 +1906-05-05 12:34:56.1537,1906-12-25 +1906-05-05 12:34:56.1538,1906-12-25 +1906-05-05 12:34:56.1539,1906-12-25 +1906-05-05 12:34:56.154,1906-12-25 +1906-05-05 12:34:56.1541,1906-12-25 +1906-05-05 12:34:56.1542,1906-12-25 +1906-05-05 12:34:56.1543,1906-12-25 +1906-05-05 12:34:56.1544,1906-12-25 +1906-05-05 12:34:56.1545,1906-12-25 +1906-05-05 12:34:56.1546,1906-12-25 +1906-05-05 12:34:56.1547,1906-12-25 +1906-05-05 12:34:56.1548,1906-12-25 +1906-05-05 12:34:56.1549,1906-12-25 +1906-05-05 12:34:56.155,1906-12-25 +1906-05-05 12:34:56.1551,1906-12-25 +1906-05-05 12:34:56.1552,1906-12-25 +1906-05-05 12:34:56.1553,1906-12-25 +1906-05-05 12:34:56.1554,1906-12-25 +1906-05-05 12:34:56.1555,1906-12-25 +1906-05-05 12:34:56.1556,1906-12-25 +1906-05-05 12:34:56.1557,1906-12-25 +1906-05-05 12:34:56.1558,1906-12-25 +1906-05-05 12:34:56.1559,1906-12-25 +1906-05-05 12:34:56.156,1906-12-25 +1906-05-05 12:34:56.1561,1906-12-25 +1906-05-05 12:34:56.1562,1906-12-25 +1906-05-05 12:34:56.1563,1906-12-25 +1906-05-05 12:34:56.1564,1906-12-25 +1906-05-05 12:34:56.1565,1906-12-25 +1906-05-05 12:34:56.1566,1906-12-25 +1906-05-05 12:34:56.1567,1906-12-25 +1906-05-05 12:34:56.1568,1906-12-25 +1906-05-05 12:34:56.1569,1906-12-25 +1906-05-05 12:34:56.157,1906-12-25 +1906-05-05 12:34:56.1571,1906-12-25 +1906-05-05 12:34:56.1572,1906-12-25 +1906-05-05 12:34:56.1573,1906-12-25 +1906-05-05 12:34:56.1574,1906-12-25 +1906-05-05 12:34:56.1575,1906-12-25 +1906-05-05 12:34:56.1576,1906-12-25 +1906-05-05 12:34:56.1577,1906-12-25 +1906-05-05 12:34:56.1578,1906-12-25 +1906-05-05 12:34:56.1579,1906-12-25 +1906-05-05 12:34:56.158,1906-12-25 +1906-05-05 12:34:56.1581,1906-12-25 +1906-05-05 12:34:56.1582,1906-12-25 +1906-05-05 12:34:56.1583,1906-12-25 +1906-05-05 12:34:56.1584,1906-12-25 +1906-05-05 12:34:56.1585,1906-12-25 +1906-05-05 12:34:56.1586,1906-12-25 +1906-05-05 12:34:56.1587,1906-12-25 +1906-05-05 12:34:56.1588,1906-12-25 +1906-05-05 12:34:56.1589,1906-12-25 +1906-05-05 12:34:56.159,1906-12-25 +1906-05-05 12:34:56.1591,1906-12-25 +1906-05-05 12:34:56.1592,1906-12-25 +1906-05-05 12:34:56.1593,1906-12-25 +1906-05-05 12:34:56.1594,1906-12-25 +1906-05-05 12:34:56.1595,1906-12-25 +1906-05-05 12:34:56.1596,1906-12-25 +1906-05-05 12:34:56.1597,1906-12-25 +1906-05-05 12:34:56.1598,1906-12-25 +1906-05-05 12:34:56.1599,1906-12-25 +1906-05-05 12:34:56.16,1906-12-25 +1906-05-05 12:34:56.1601,1906-12-25 +1906-05-05 12:34:56.1602,1906-12-25 +1906-05-05 12:34:56.1603,1906-12-25 +1906-05-05 12:34:56.1604,1906-12-25 +1906-05-05 12:34:56.1605,1906-12-25 +1906-05-05 12:34:56.1606,1906-12-25 +1906-05-05 12:34:56.1607,1906-12-25 +1906-05-05 12:34:56.1608,1906-12-25 +1906-05-05 12:34:56.1609,1906-12-25 +1906-05-05 12:34:56.161,1906-12-25 +1906-05-05 12:34:56.1611,1906-12-25 +1906-05-05 12:34:56.1612,1906-12-25 +1906-05-05 12:34:56.1613,1906-12-25 +1906-05-05 12:34:56.1614,1906-12-25 +1906-05-05 12:34:56.1615,1906-12-25 +1906-05-05 12:34:56.1616,1906-12-25 +1906-05-05 12:34:56.1617,1906-12-25 +1906-05-05 12:34:56.1618,1906-12-25 +1906-05-05 12:34:56.1619,1906-12-25 +1906-05-05 12:34:56.162,1906-12-25 +1906-05-05 12:34:56.1621,1906-12-25 +1906-05-05 12:34:56.1622,1906-12-25 +1906-05-05 12:34:56.1623,1906-12-25 +1906-05-05 12:34:56.1624,1906-12-25 +1906-05-05 12:34:56.1625,1906-12-25 +1906-05-05 12:34:56.1626,1906-12-25 +1906-05-05 12:34:56.1627,1906-12-25 +1906-05-05 12:34:56.1628,1906-12-25 +1906-05-05 12:34:56.1629,1906-12-25 +1906-05-05 12:34:56.163,1906-12-25 +1906-05-05 12:34:56.1631,1906-12-25 +1906-05-05 12:34:56.1632,1906-12-25 +1906-05-05 12:34:56.1633,1906-12-25 +1906-05-05 12:34:56.1634,1906-12-25 +1906-05-05 12:34:56.1635,1906-12-25 +1906-05-05 12:34:56.1636,1906-12-25 +1906-05-05 12:34:56.1637,1906-12-25 +1906-05-05 12:34:56.1638,1906-12-25 +1906-05-05 12:34:56.1639,1906-12-25 +1906-05-05 12:34:56.164,1906-12-25 +1906-05-05 12:34:56.1641,1906-12-25 +1906-05-05 12:34:56.1642,1906-12-25 +1906-05-05 12:34:56.1643,1906-12-25 +1906-05-05 12:34:56.1644,1906-12-25 +1906-05-05 12:34:56.1645,1906-12-25 +1906-05-05 12:34:56.1646,1906-12-25 +1906-05-05 12:34:56.1647,1906-12-25 +1906-05-05 12:34:56.1648,1906-12-25 +1906-05-05 12:34:56.1649,1906-12-25 +1906-05-05 12:34:56.165,1906-12-25 +1906-05-05 12:34:56.1651,1906-12-25 +1906-05-05 12:34:56.1652,1906-12-25 +1906-05-05 12:34:56.1653,1906-12-25 +1906-05-05 12:34:56.1654,1906-12-25 +1906-05-05 12:34:56.1655,1906-12-25 +1906-05-05 12:34:56.1656,1906-12-25 +1906-05-05 12:34:56.1657,1906-12-25 +1906-05-05 12:34:56.1658,1906-12-25 +1906-05-05 12:34:56.1659,1906-12-25 +1906-05-05 12:34:56.166,1906-12-25 +1906-05-05 12:34:56.1661,1906-12-25 +1906-05-05 12:34:56.1662,1906-12-25 +1906-05-05 12:34:56.1663,1906-12-25 +1906-05-05 12:34:56.1664,1906-12-25 +1906-05-05 12:34:56.1665,1906-12-25 +1906-05-05 12:34:56.1666,1906-12-25 +1906-05-05 12:34:56.1667,1906-12-25 +1906-05-05 12:34:56.1668,1906-12-25 +1906-05-05 12:34:56.1669,1906-12-25 +1906-05-05 12:34:56.167,1906-12-25 +1906-05-05 12:34:56.1671,1906-12-25 +1906-05-05 12:34:56.1672,1906-12-25 +1906-05-05 12:34:56.1673,1906-12-25 +1906-05-05 12:34:56.1674,1906-12-25 +1906-05-05 12:34:56.1675,1906-12-25 +1906-05-05 12:34:56.1676,1906-12-25 +1906-05-05 12:34:56.1677,1906-12-25 +1906-05-05 12:34:56.1678,1906-12-25 +1906-05-05 12:34:56.1679,1906-12-25 +1906-05-05 12:34:56.168,1906-12-25 +1906-05-05 12:34:56.1681,1906-12-25 +1906-05-05 12:34:56.1682,1906-12-25 +1906-05-05 12:34:56.1683,1906-12-25 +1906-05-05 12:34:56.1684,1906-12-25 +1906-05-05 12:34:56.1685,1906-12-25 +1906-05-05 12:34:56.1686,1906-12-25 +1906-05-05 12:34:56.1687,1906-12-25 +1906-05-05 12:34:56.1688,1906-12-25 +1906-05-05 12:34:56.1689,1906-12-25 +1906-05-05 12:34:56.169,1906-12-25 +1906-05-05 12:34:56.1691,1906-12-25 +1906-05-05 12:34:56.1692,1906-12-25 +1906-05-05 12:34:56.1693,1906-12-25 +1906-05-05 12:34:56.1694,1906-12-25 +1906-05-05 12:34:56.1695,1906-12-25 +1906-05-05 12:34:56.1696,1906-12-25 +1906-05-05 12:34:56.1697,1906-12-25 +1906-05-05 12:34:56.1698,1906-12-25 +1906-05-05 12:34:56.1699,1906-12-25 +1906-05-05 12:34:56.17,1906-12-25 +1906-05-05 12:34:56.1701,1906-12-25 +1906-05-05 12:34:56.1702,1906-12-25 +1906-05-05 12:34:56.1703,1906-12-25 +1906-05-05 12:34:56.1704,1906-12-25 +1906-05-05 12:34:56.1705,1906-12-25 +1906-05-05 12:34:56.1706,1906-12-25 +1906-05-05 12:34:56.1707,1906-12-25 +1906-05-05 12:34:56.1708,1906-12-25 +1906-05-05 12:34:56.1709,1906-12-25 +1906-05-05 12:34:56.171,1906-12-25 +1906-05-05 12:34:56.1711,1906-12-25 +1906-05-05 12:34:56.1712,1906-12-25 +1906-05-05 12:34:56.1713,1906-12-25 +1906-05-05 12:34:56.1714,1906-12-25 +1906-05-05 12:34:56.1715,1906-12-25 +1906-05-05 12:34:56.1716,1906-12-25 +1906-05-05 12:34:56.1717,1906-12-25 +1906-05-05 12:34:56.1718,1906-12-25 +1906-05-05 12:34:56.1719,1906-12-25 +1906-05-05 12:34:56.172,1906-12-25 +1906-05-05 12:34:56.1721,1906-12-25 +1906-05-05 12:34:56.1722,1906-12-25 +1906-05-05 12:34:56.1723,1906-12-25 +1906-05-05 12:34:56.1724,1906-12-25 +1906-05-05 12:34:56.1725,1906-12-25 +1906-05-05 12:34:56.1726,1906-12-25 +1906-05-05 12:34:56.1727,1906-12-25 +1906-05-05 12:34:56.1728,1906-12-25 +1906-05-05 12:34:56.1729,1906-12-25 +1906-05-05 12:34:56.173,1906-12-25 +1906-05-05 12:34:56.1731,1906-12-25 +1906-05-05 12:34:56.1732,1906-12-25 +1906-05-05 12:34:56.1733,1906-12-25 +1906-05-05 12:34:56.1734,1906-12-25 +1906-05-05 12:34:56.1735,1906-12-25 +1906-05-05 12:34:56.1736,1906-12-25 +1906-05-05 12:34:56.1737,1906-12-25 +1906-05-05 12:34:56.1738,1906-12-25 +1906-05-05 12:34:56.1739,1906-12-25 +1906-05-05 12:34:56.174,1906-12-25 +1906-05-05 12:34:56.1741,1906-12-25 +1906-05-05 12:34:56.1742,1906-12-25 +1906-05-05 12:34:56.1743,1906-12-25 +1906-05-05 12:34:56.1744,1906-12-25 +1906-05-05 12:34:56.1745,1906-12-25 +1906-05-05 12:34:56.1746,1906-12-25 +1906-05-05 12:34:56.1747,1906-12-25 +1906-05-05 12:34:56.1748,1906-12-25 +1906-05-05 12:34:56.1749,1906-12-25 +1906-05-05 12:34:56.175,1906-12-25 +1906-05-05 12:34:56.1751,1906-12-25 +1906-05-05 12:34:56.1752,1906-12-25 +1906-05-05 12:34:56.1753,1906-12-25 +1906-05-05 12:34:56.1754,1906-12-25 +1906-05-05 12:34:56.1755,1906-12-25 +1906-05-05 12:34:56.1756,1906-12-25 +1906-05-05 12:34:56.1757,1906-12-25 +1906-05-05 12:34:56.1758,1906-12-25 +1906-05-05 12:34:56.1759,1906-12-25 +1906-05-05 12:34:56.176,1906-12-25 +1906-05-05 12:34:56.1761,1906-12-25 +1906-05-05 12:34:56.1762,1906-12-25 +1906-05-05 12:34:56.1763,1906-12-25 +1906-05-05 12:34:56.1764,1906-12-25 +1906-05-05 12:34:56.1765,1906-12-25 +1906-05-05 12:34:56.1766,1906-12-25 +1906-05-05 12:34:56.1767,1906-12-25 +1906-05-05 12:34:56.1768,1906-12-25 +1906-05-05 12:34:56.1769,1906-12-25 +1906-05-05 12:34:56.177,1906-12-25 +1906-05-05 12:34:56.1771,1906-12-25 +1906-05-05 12:34:56.1772,1906-12-25 +1906-05-05 12:34:56.1773,1906-12-25 +1906-05-05 12:34:56.1774,1906-12-25 +1906-05-05 12:34:56.1775,1906-12-25 +1906-05-05 12:34:56.1776,1906-12-25 +1906-05-05 12:34:56.1777,1906-12-25 +1906-05-05 12:34:56.1778,1906-12-25 +1906-05-05 12:34:56.1779,1906-12-25 +1906-05-05 12:34:56.178,1906-12-25 +1906-05-05 12:34:56.1781,1906-12-25 +1906-05-05 12:34:56.1782,1906-12-25 +1906-05-05 12:34:56.1783,1906-12-25 +1906-05-05 12:34:56.1784,1906-12-25 +1906-05-05 12:34:56.1785,1906-12-25 +1906-05-05 12:34:56.1786,1906-12-25 +1906-05-05 12:34:56.1787,1906-12-25 +1906-05-05 12:34:56.1788,1906-12-25 +1906-05-05 12:34:56.1789,1906-12-25 +1906-05-05 12:34:56.179,1906-12-25 +1906-05-05 12:34:56.1791,1906-12-25 +1906-05-05 12:34:56.1792,1906-12-25 +1906-05-05 12:34:56.1793,1906-12-25 +1906-05-05 12:34:56.1794,1906-12-25 +1906-05-05 12:34:56.1795,1906-12-25 +1906-05-05 12:34:56.1796,1906-12-25 +1906-05-05 12:34:56.1797,1906-12-25 +1906-05-05 12:34:56.1798,1906-12-25 +1906-05-05 12:34:56.1799,1906-12-25 +1906-05-05 12:34:56.18,1906-12-25 +1906-05-05 12:34:56.1801,1906-12-25 +1906-05-05 12:34:56.1802,1906-12-25 +1906-05-05 12:34:56.1803,1906-12-25 +1906-05-05 12:34:56.1804,1906-12-25 +1906-05-05 12:34:56.1805,1906-12-25 +1906-05-05 12:34:56.1806,1906-12-25 +1906-05-05 12:34:56.1807,1906-12-25 +1906-05-05 12:34:56.1808,1906-12-25 +1906-05-05 12:34:56.1809,1906-12-25 +1906-05-05 12:34:56.181,1906-12-25 +1906-05-05 12:34:56.1811,1906-12-25 +1906-05-05 12:34:56.1812,1906-12-25 +1906-05-05 12:34:56.1813,1906-12-25 +1906-05-05 12:34:56.1814,1906-12-25 +1906-05-05 12:34:56.1815,1906-12-25 +1906-05-05 12:34:56.1816,1906-12-25 +1906-05-05 12:34:56.1817,1906-12-25 +1906-05-05 12:34:56.1818,1906-12-25 +1906-05-05 12:34:56.1819,1906-12-25 +1906-05-05 12:34:56.182,1906-12-25 +1906-05-05 12:34:56.1821,1906-12-25 +1906-05-05 12:34:56.1822,1906-12-25 +1906-05-05 12:34:56.1823,1906-12-25 +1906-05-05 12:34:56.1824,1906-12-25 +1906-05-05 12:34:56.1825,1906-12-25 +1906-05-05 12:34:56.1826,1906-12-25 +1906-05-05 12:34:56.1827,1906-12-25 +1906-05-05 12:34:56.1828,1906-12-25 +1906-05-05 12:34:56.1829,1906-12-25 +1906-05-05 12:34:56.183,1906-12-25 +1906-05-05 12:34:56.1831,1906-12-25 +1906-05-05 12:34:56.1832,1906-12-25 +1906-05-05 12:34:56.1833,1906-12-25 +1906-05-05 12:34:56.1834,1906-12-25 +1906-05-05 12:34:56.1835,1906-12-25 +1906-05-05 12:34:56.1836,1906-12-25 +1906-05-05 12:34:56.1837,1906-12-25 +1906-05-05 12:34:56.1838,1906-12-25 +1906-05-05 12:34:56.1839,1906-12-25 +1906-05-05 12:34:56.184,1906-12-25 +1906-05-05 12:34:56.1841,1906-12-25 +1906-05-05 12:34:56.1842,1906-12-25 +1906-05-05 12:34:56.1843,1906-12-25 +1906-05-05 12:34:56.1844,1906-12-25 +1906-05-05 12:34:56.1845,1906-12-25 +1906-05-05 12:34:56.1846,1906-12-25 +1906-05-05 12:34:56.1847,1906-12-25 +1906-05-05 12:34:56.1848,1906-12-25 +1906-05-05 12:34:56.1849,1906-12-25 +1906-05-05 12:34:56.185,1906-12-25 +1906-05-05 12:34:56.1851,1906-12-25 +1906-05-05 12:34:56.1852,1906-12-25 +1906-05-05 12:34:56.1853,1906-12-25 +1906-05-05 12:34:56.1854,1906-12-25 +1906-05-05 12:34:56.1855,1906-12-25 +1906-05-05 12:34:56.1856,1906-12-25 +1906-05-05 12:34:56.1857,1906-12-25 +1906-05-05 12:34:56.1858,1906-12-25 +1906-05-05 12:34:56.1859,1906-12-25 +1906-05-05 12:34:56.186,1906-12-25 +1906-05-05 12:34:56.1861,1906-12-25 +1906-05-05 12:34:56.1862,1906-12-25 +1906-05-05 12:34:56.1863,1906-12-25 +1906-05-05 12:34:56.1864,1906-12-25 +1906-05-05 12:34:56.1865,1906-12-25 +1906-05-05 12:34:56.1866,1906-12-25 +1906-05-05 12:34:56.1867,1906-12-25 +1906-05-05 12:34:56.1868,1906-12-25 +1906-05-05 12:34:56.1869,1906-12-25 +1906-05-05 12:34:56.187,1906-12-25 +1906-05-05 12:34:56.1871,1906-12-25 +1906-05-05 12:34:56.1872,1906-12-25 +1906-05-05 12:34:56.1873,1906-12-25 +1906-05-05 12:34:56.1874,1906-12-25 +1906-05-05 12:34:56.1875,1906-12-25 +1906-05-05 12:34:56.1876,1906-12-25 +1906-05-05 12:34:56.1877,1906-12-25 +1906-05-05 12:34:56.1878,1906-12-25 +1906-05-05 12:34:56.1879,1906-12-25 +1906-05-05 12:34:56.188,1906-12-25 +1906-05-05 12:34:56.1881,1906-12-25 +1906-05-05 12:34:56.1882,1906-12-25 +1906-05-05 12:34:56.1883,1906-12-25 +1906-05-05 12:34:56.1884,1906-12-25 +1906-05-05 12:34:56.1885,1906-12-25 +1906-05-05 12:34:56.1886,1906-12-25 +1906-05-05 12:34:56.1887,1906-12-25 +1906-05-05 12:34:56.1888,1906-12-25 +1906-05-05 12:34:56.1889,1906-12-25 +1906-05-05 12:34:56.189,1906-12-25 +1906-05-05 12:34:56.1891,1906-12-25 +1906-05-05 12:34:56.1892,1906-12-25 +1906-05-05 12:34:56.1893,1906-12-25 +1906-05-05 12:34:56.1894,1906-12-25 +1906-05-05 12:34:56.1895,1906-12-25 +1906-05-05 12:34:56.1896,1906-12-25 +1906-05-05 12:34:56.1897,1906-12-25 +1906-05-05 12:34:56.1898,1906-12-25 +1906-05-05 12:34:56.1899,1906-12-25 +1906-05-05 12:34:56.19,1906-12-25 +1906-05-05 12:34:56.1901,1906-12-25 +1906-05-05 12:34:56.1902,1906-12-25 +1906-05-05 12:34:56.1903,1906-12-25 +1906-05-05 12:34:56.1904,1906-12-25 +1906-05-05 12:34:56.1905,1906-12-25 +1906-05-05 12:34:56.1906,1906-12-25 +1906-05-05 12:34:56.1907,1906-12-25 +1906-05-05 12:34:56.1908,1906-12-25 +1906-05-05 12:34:56.1909,1906-12-25 +1906-05-05 12:34:56.191,1906-12-25 +1906-05-05 12:34:56.1911,1906-12-25 +1906-05-05 12:34:56.1912,1906-12-25 +1906-05-05 12:34:56.1913,1906-12-25 +1906-05-05 12:34:56.1914,1906-12-25 +1906-05-05 12:34:56.1915,1906-12-25 +1906-05-05 12:34:56.1916,1906-12-25 +1906-05-05 12:34:56.1917,1906-12-25 +1906-05-05 12:34:56.1918,1906-12-25 +1906-05-05 12:34:56.1919,1906-12-25 +1906-05-05 12:34:56.192,1906-12-25 +1906-05-05 12:34:56.1921,1906-12-25 +1906-05-05 12:34:56.1922,1906-12-25 +1906-05-05 12:34:56.1923,1906-12-25 +1906-05-05 12:34:56.1924,1906-12-25 +1906-05-05 12:34:56.1925,1906-12-25 +1906-05-05 12:34:56.1926,1906-12-25 +1906-05-05 12:34:56.1927,1906-12-25 +1906-05-05 12:34:56.1928,1906-12-25 +1906-05-05 12:34:56.1929,1906-12-25 +1906-05-05 12:34:56.193,1906-12-25 +1906-05-05 12:34:56.1931,1906-12-25 +1906-05-05 12:34:56.1932,1906-12-25 +1906-05-05 12:34:56.1933,1906-12-25 +1906-05-05 12:34:56.1934,1906-12-25 +1906-05-05 12:34:56.1935,1906-12-25 +1906-05-05 12:34:56.1936,1906-12-25 +1906-05-05 12:34:56.1937,1906-12-25 +1906-05-05 12:34:56.1938,1906-12-25 +1906-05-05 12:34:56.1939,1906-12-25 +1906-05-05 12:34:56.194,1906-12-25 +1906-05-05 12:34:56.1941,1906-12-25 +1906-05-05 12:34:56.1942,1906-12-25 +1906-05-05 12:34:56.1943,1906-12-25 +1906-05-05 12:34:56.1944,1906-12-25 +1906-05-05 12:34:56.1945,1906-12-25 +1906-05-05 12:34:56.1946,1906-12-25 +1906-05-05 12:34:56.1947,1906-12-25 +1906-05-05 12:34:56.1948,1906-12-25 +1906-05-05 12:34:56.1949,1906-12-25 +1906-05-05 12:34:56.195,1906-12-25 +1906-05-05 12:34:56.1951,1906-12-25 +1906-05-05 12:34:56.1952,1906-12-25 +1906-05-05 12:34:56.1953,1906-12-25 +1906-05-05 12:34:56.1954,1906-12-25 +1906-05-05 12:34:56.1955,1906-12-25 +1906-05-05 12:34:56.1956,1906-12-25 +1906-05-05 12:34:56.1957,1906-12-25 +1906-05-05 12:34:56.1958,1906-12-25 +1906-05-05 12:34:56.1959,1906-12-25 +1906-05-05 12:34:56.196,1906-12-25 +1906-05-05 12:34:56.1961,1906-12-25 +1906-05-05 12:34:56.1962,1906-12-25 +1906-05-05 12:34:56.1963,1906-12-25 +1906-05-05 12:34:56.1964,1906-12-25 +1906-05-05 12:34:56.1965,1906-12-25 +1906-05-05 12:34:56.1966,1906-12-25 +1906-05-05 12:34:56.1967,1906-12-25 +1906-05-05 12:34:56.1968,1906-12-25 +1906-05-05 12:34:56.1969,1906-12-25 +1906-05-05 12:34:56.197,1906-12-25 +1906-05-05 12:34:56.1971,1906-12-25 +1906-05-05 12:34:56.1972,1906-12-25 +1906-05-05 12:34:56.1973,1906-12-25 +1906-05-05 12:34:56.1974,1906-12-25 +1906-05-05 12:34:56.1975,1906-12-25 +1906-05-05 12:34:56.1976,1906-12-25 +1906-05-05 12:34:56.1977,1906-12-25 +1906-05-05 12:34:56.1978,1906-12-25 +1906-05-05 12:34:56.1979,1906-12-25 +1906-05-05 12:34:56.198,1906-12-25 +1906-05-05 12:34:56.1981,1906-12-25 +1906-05-05 12:34:56.1982,1906-12-25 +1906-05-05 12:34:56.1983,1906-12-25 +1906-05-05 12:34:56.1984,1906-12-25 +1906-05-05 12:34:56.1985,1906-12-25 +1906-05-05 12:34:56.1986,1906-12-25 +1906-05-05 12:34:56.1987,1906-12-25 +1906-05-05 12:34:56.1988,1906-12-25 +1906-05-05 12:34:56.1989,1906-12-25 +1906-05-05 12:34:56.199,1906-12-25 +1906-05-05 12:34:56.1991,1906-12-25 +1906-05-05 12:34:56.1992,1906-12-25 +1906-05-05 12:34:56.1993,1906-12-25 +1906-05-05 12:34:56.1994,1906-12-25 +1906-05-05 12:34:56.1995,1906-12-25 +1906-05-05 12:34:56.1996,1906-12-25 +1906-05-05 12:34:56.1997,1906-12-25 +1906-05-05 12:34:56.1998,1906-12-25 +1906-05-05 12:34:56.1999,1906-12-25 +1907-05-05 12:34:56.1,1907-12-25 +1907-05-05 12:34:56.1001,1907-12-25 +1907-05-05 12:34:56.1002,1907-12-25 +1907-05-05 12:34:56.1003,1907-12-25 +1907-05-05 12:34:56.1004,1907-12-25 +1907-05-05 12:34:56.1005,1907-12-25 +1907-05-05 12:34:56.1006,1907-12-25 +1907-05-05 12:34:56.1007,1907-12-25 +1907-05-05 12:34:56.1008,1907-12-25 +1907-05-05 12:34:56.1009,1907-12-25 +1907-05-05 12:34:56.101,1907-12-25 +1907-05-05 12:34:56.1011,1907-12-25 +1907-05-05 12:34:56.1012,1907-12-25 +1907-05-05 12:34:56.1013,1907-12-25 +1907-05-05 12:34:56.1014,1907-12-25 +1907-05-05 12:34:56.1015,1907-12-25 +1907-05-05 12:34:56.1016,1907-12-25 +1907-05-05 12:34:56.1017,1907-12-25 +1907-05-05 12:34:56.1018,1907-12-25 +1907-05-05 12:34:56.1019,1907-12-25 +1907-05-05 12:34:56.102,1907-12-25 +1907-05-05 12:34:56.1021,1907-12-25 +1907-05-05 12:34:56.1022,1907-12-25 +1907-05-05 12:34:56.1023,1907-12-25 +1907-05-05 12:34:56.1024,1907-12-25 +1907-05-05 12:34:56.1025,1907-12-25 +1907-05-05 12:34:56.1026,1907-12-25 +1907-05-05 12:34:56.1027,1907-12-25 +1907-05-05 12:34:56.1028,1907-12-25 +1907-05-05 12:34:56.1029,1907-12-25 +1907-05-05 12:34:56.103,1907-12-25 +1907-05-05 12:34:56.1031,1907-12-25 +1907-05-05 12:34:56.1032,1907-12-25 +1907-05-05 12:34:56.1033,1907-12-25 +1907-05-05 12:34:56.1034,1907-12-25 +1907-05-05 12:34:56.1035,1907-12-25 +1907-05-05 12:34:56.1036,1907-12-25 +1907-05-05 12:34:56.1037,1907-12-25 +1907-05-05 12:34:56.1038,1907-12-25 +1907-05-05 12:34:56.1039,1907-12-25 +1907-05-05 12:34:56.104,1907-12-25 +1907-05-05 12:34:56.1041,1907-12-25 +1907-05-05 12:34:56.1042,1907-12-25 +1907-05-05 12:34:56.1043,1907-12-25 +1907-05-05 12:34:56.1044,1907-12-25 +1907-05-05 12:34:56.1045,1907-12-25 +1907-05-05 12:34:56.1046,1907-12-25 +1907-05-05 12:34:56.1047,1907-12-25 +1907-05-05 12:34:56.1048,1907-12-25 +1907-05-05 12:34:56.1049,1907-12-25 +1907-05-05 12:34:56.105,1907-12-25 +1907-05-05 12:34:56.1051,1907-12-25 +1907-05-05 12:34:56.1052,1907-12-25 +1907-05-05 12:34:56.1053,1907-12-25 +1907-05-05 12:34:56.1054,1907-12-25 +1907-05-05 12:34:56.1055,1907-12-25 +1907-05-05 12:34:56.1056,1907-12-25 +1907-05-05 12:34:56.1057,1907-12-25 +1907-05-05 12:34:56.1058,1907-12-25 +1907-05-05 12:34:56.1059,1907-12-25 +1907-05-05 12:34:56.106,1907-12-25 +1907-05-05 12:34:56.1061,1907-12-25 +1907-05-05 12:34:56.1062,1907-12-25 +1907-05-05 12:34:56.1063,1907-12-25 +1907-05-05 12:34:56.1064,1907-12-25 +1907-05-05 12:34:56.1065,1907-12-25 +1907-05-05 12:34:56.1066,1907-12-25 +1907-05-05 12:34:56.1067,1907-12-25 +1907-05-05 12:34:56.1068,1907-12-25 +1907-05-05 12:34:56.1069,1907-12-25 +1907-05-05 12:34:56.107,1907-12-25 +1907-05-05 12:34:56.1071,1907-12-25 +1907-05-05 12:34:56.1072,1907-12-25 +1907-05-05 12:34:56.1073,1907-12-25 +1907-05-05 12:34:56.1074,1907-12-25 +1907-05-05 12:34:56.1075,1907-12-25 +1907-05-05 12:34:56.1076,1907-12-25 +1907-05-05 12:34:56.1077,1907-12-25 +1907-05-05 12:34:56.1078,1907-12-25 +1907-05-05 12:34:56.1079,1907-12-25 +1907-05-05 12:34:56.108,1907-12-25 +1907-05-05 12:34:56.1081,1907-12-25 +1907-05-05 12:34:56.1082,1907-12-25 +1907-05-05 12:34:56.1083,1907-12-25 +1907-05-05 12:34:56.1084,1907-12-25 +1907-05-05 12:34:56.1085,1907-12-25 +1907-05-05 12:34:56.1086,1907-12-25 +1907-05-05 12:34:56.1087,1907-12-25 +1907-05-05 12:34:56.1088,1907-12-25 +1907-05-05 12:34:56.1089,1907-12-25 +1907-05-05 12:34:56.109,1907-12-25 +1907-05-05 12:34:56.1091,1907-12-25 +1907-05-05 12:34:56.1092,1907-12-25 +1907-05-05 12:34:56.1093,1907-12-25 +1907-05-05 12:34:56.1094,1907-12-25 +1907-05-05 12:34:56.1095,1907-12-25 +1907-05-05 12:34:56.1096,1907-12-25 +1907-05-05 12:34:56.1097,1907-12-25 +1907-05-05 12:34:56.1098,1907-12-25 +1907-05-05 12:34:56.1099,1907-12-25 +1907-05-05 12:34:56.11,1907-12-25 +1907-05-05 12:34:56.1101,1907-12-25 +1907-05-05 12:34:56.1102,1907-12-25 +1907-05-05 12:34:56.1103,1907-12-25 +1907-05-05 12:34:56.1104,1907-12-25 +1907-05-05 12:34:56.1105,1907-12-25 +1907-05-05 12:34:56.1106,1907-12-25 +1907-05-05 12:34:56.1107,1907-12-25 +1907-05-05 12:34:56.1108,1907-12-25 +1907-05-05 12:34:56.1109,1907-12-25 +1907-05-05 12:34:56.111,1907-12-25 +1907-05-05 12:34:56.1111,1907-12-25 +1907-05-05 12:34:56.1112,1907-12-25 +1907-05-05 12:34:56.1113,1907-12-25 +1907-05-05 12:34:56.1114,1907-12-25 +1907-05-05 12:34:56.1115,1907-12-25 +1907-05-05 12:34:56.1116,1907-12-25 +1907-05-05 12:34:56.1117,1907-12-25 +1907-05-05 12:34:56.1118,1907-12-25 +1907-05-05 12:34:56.1119,1907-12-25 +1907-05-05 12:34:56.112,1907-12-25 +1907-05-05 12:34:56.1121,1907-12-25 +1907-05-05 12:34:56.1122,1907-12-25 +1907-05-05 12:34:56.1123,1907-12-25 +1907-05-05 12:34:56.1124,1907-12-25 +1907-05-05 12:34:56.1125,1907-12-25 +1907-05-05 12:34:56.1126,1907-12-25 +1907-05-05 12:34:56.1127,1907-12-25 +1907-05-05 12:34:56.1128,1907-12-25 +1907-05-05 12:34:56.1129,1907-12-25 +1907-05-05 12:34:56.113,1907-12-25 +1907-05-05 12:34:56.1131,1907-12-25 +1907-05-05 12:34:56.1132,1907-12-25 +1907-05-05 12:34:56.1133,1907-12-25 +1907-05-05 12:34:56.1134,1907-12-25 +1907-05-05 12:34:56.1135,1907-12-25 +1907-05-05 12:34:56.1136,1907-12-25 +1907-05-05 12:34:56.1137,1907-12-25 +1907-05-05 12:34:56.1138,1907-12-25 +1907-05-05 12:34:56.1139,1907-12-25 +1907-05-05 12:34:56.114,1907-12-25 +1907-05-05 12:34:56.1141,1907-12-25 +1907-05-05 12:34:56.1142,1907-12-25 +1907-05-05 12:34:56.1143,1907-12-25 +1907-05-05 12:34:56.1144,1907-12-25 +1907-05-05 12:34:56.1145,1907-12-25 +1907-05-05 12:34:56.1146,1907-12-25 +1907-05-05 12:34:56.1147,1907-12-25 +1907-05-05 12:34:56.1148,1907-12-25 +1907-05-05 12:34:56.1149,1907-12-25 +1907-05-05 12:34:56.115,1907-12-25 +1907-05-05 12:34:56.1151,1907-12-25 +1907-05-05 12:34:56.1152,1907-12-25 +1907-05-05 12:34:56.1153,1907-12-25 +1907-05-05 12:34:56.1154,1907-12-25 +1907-05-05 12:34:56.1155,1907-12-25 +1907-05-05 12:34:56.1156,1907-12-25 +1907-05-05 12:34:56.1157,1907-12-25 +1907-05-05 12:34:56.1158,1907-12-25 +1907-05-05 12:34:56.1159,1907-12-25 +1907-05-05 12:34:56.116,1907-12-25 +1907-05-05 12:34:56.1161,1907-12-25 +1907-05-05 12:34:56.1162,1907-12-25 +1907-05-05 12:34:56.1163,1907-12-25 +1907-05-05 12:34:56.1164,1907-12-25 +1907-05-05 12:34:56.1165,1907-12-25 +1907-05-05 12:34:56.1166,1907-12-25 +1907-05-05 12:34:56.1167,1907-12-25 +1907-05-05 12:34:56.1168,1907-12-25 +1907-05-05 12:34:56.1169,1907-12-25 +1907-05-05 12:34:56.117,1907-12-25 +1907-05-05 12:34:56.1171,1907-12-25 +1907-05-05 12:34:56.1172,1907-12-25 +1907-05-05 12:34:56.1173,1907-12-25 +1907-05-05 12:34:56.1174,1907-12-25 +1907-05-05 12:34:56.1175,1907-12-25 +1907-05-05 12:34:56.1176,1907-12-25 +1907-05-05 12:34:56.1177,1907-12-25 +1907-05-05 12:34:56.1178,1907-12-25 +1907-05-05 12:34:56.1179,1907-12-25 +1907-05-05 12:34:56.118,1907-12-25 +1907-05-05 12:34:56.1181,1907-12-25 +1907-05-05 12:34:56.1182,1907-12-25 +1907-05-05 12:34:56.1183,1907-12-25 +1907-05-05 12:34:56.1184,1907-12-25 +1907-05-05 12:34:56.1185,1907-12-25 +1907-05-05 12:34:56.1186,1907-12-25 +1907-05-05 12:34:56.1187,1907-12-25 +1907-05-05 12:34:56.1188,1907-12-25 +1907-05-05 12:34:56.1189,1907-12-25 +1907-05-05 12:34:56.119,1907-12-25 +1907-05-05 12:34:56.1191,1907-12-25 +1907-05-05 12:34:56.1192,1907-12-25 +1907-05-05 12:34:56.1193,1907-12-25 +1907-05-05 12:34:56.1194,1907-12-25 +1907-05-05 12:34:56.1195,1907-12-25 +1907-05-05 12:34:56.1196,1907-12-25 +1907-05-05 12:34:56.1197,1907-12-25 +1907-05-05 12:34:56.1198,1907-12-25 +1907-05-05 12:34:56.1199,1907-12-25 +1907-05-05 12:34:56.12,1907-12-25 +1907-05-05 12:34:56.1201,1907-12-25 +1907-05-05 12:34:56.1202,1907-12-25 +1907-05-05 12:34:56.1203,1907-12-25 +1907-05-05 12:34:56.1204,1907-12-25 +1907-05-05 12:34:56.1205,1907-12-25 +1907-05-05 12:34:56.1206,1907-12-25 +1907-05-05 12:34:56.1207,1907-12-25 +1907-05-05 12:34:56.1208,1907-12-25 +1907-05-05 12:34:56.1209,1907-12-25 +1907-05-05 12:34:56.121,1907-12-25 +1907-05-05 12:34:56.1211,1907-12-25 +1907-05-05 12:34:56.1212,1907-12-25 +1907-05-05 12:34:56.1213,1907-12-25 +1907-05-05 12:34:56.1214,1907-12-25 +1907-05-05 12:34:56.1215,1907-12-25 +1907-05-05 12:34:56.1216,1907-12-25 +1907-05-05 12:34:56.1217,1907-12-25 +1907-05-05 12:34:56.1218,1907-12-25 +1907-05-05 12:34:56.1219,1907-12-25 +1907-05-05 12:34:56.122,1907-12-25 +1907-05-05 12:34:56.1221,1907-12-25 +1907-05-05 12:34:56.1222,1907-12-25 +1907-05-05 12:34:56.1223,1907-12-25 +1907-05-05 12:34:56.1224,1907-12-25 +1907-05-05 12:34:56.1225,1907-12-25 +1907-05-05 12:34:56.1226,1907-12-25 +1907-05-05 12:34:56.1227,1907-12-25 +1907-05-05 12:34:56.1228,1907-12-25 +1907-05-05 12:34:56.1229,1907-12-25 +1907-05-05 12:34:56.123,1907-12-25 +1907-05-05 12:34:56.1231,1907-12-25 +1907-05-05 12:34:56.1232,1907-12-25 +1907-05-05 12:34:56.1233,1907-12-25 +1907-05-05 12:34:56.1234,1907-12-25 +1907-05-05 12:34:56.1235,1907-12-25 +1907-05-05 12:34:56.1236,1907-12-25 +1907-05-05 12:34:56.1237,1907-12-25 +1907-05-05 12:34:56.1238,1907-12-25 +1907-05-05 12:34:56.1239,1907-12-25 +1907-05-05 12:34:56.124,1907-12-25 +1907-05-05 12:34:56.1241,1907-12-25 +1907-05-05 12:34:56.1242,1907-12-25 +1907-05-05 12:34:56.1243,1907-12-25 +1907-05-05 12:34:56.1244,1907-12-25 +1907-05-05 12:34:56.1245,1907-12-25 +1907-05-05 12:34:56.1246,1907-12-25 +1907-05-05 12:34:56.1247,1907-12-25 +1907-05-05 12:34:56.1248,1907-12-25 +1907-05-05 12:34:56.1249,1907-12-25 +1907-05-05 12:34:56.125,1907-12-25 +1907-05-05 12:34:56.1251,1907-12-25 +1907-05-05 12:34:56.1252,1907-12-25 +1907-05-05 12:34:56.1253,1907-12-25 +1907-05-05 12:34:56.1254,1907-12-25 +1907-05-05 12:34:56.1255,1907-12-25 +1907-05-05 12:34:56.1256,1907-12-25 +1907-05-05 12:34:56.1257,1907-12-25 +1907-05-05 12:34:56.1258,1907-12-25 +1907-05-05 12:34:56.1259,1907-12-25 +1907-05-05 12:34:56.126,1907-12-25 +1907-05-05 12:34:56.1261,1907-12-25 +1907-05-05 12:34:56.1262,1907-12-25 +1907-05-05 12:34:56.1263,1907-12-25 +1907-05-05 12:34:56.1264,1907-12-25 +1907-05-05 12:34:56.1265,1907-12-25 +1907-05-05 12:34:56.1266,1907-12-25 +1907-05-05 12:34:56.1267,1907-12-25 +1907-05-05 12:34:56.1268,1907-12-25 +1907-05-05 12:34:56.1269,1907-12-25 +1907-05-05 12:34:56.127,1907-12-25 +1907-05-05 12:34:56.1271,1907-12-25 +1907-05-05 12:34:56.1272,1907-12-25 +1907-05-05 12:34:56.1273,1907-12-25 +1907-05-05 12:34:56.1274,1907-12-25 +1907-05-05 12:34:56.1275,1907-12-25 +1907-05-05 12:34:56.1276,1907-12-25 +1907-05-05 12:34:56.1277,1907-12-25 +1907-05-05 12:34:56.1278,1907-12-25 +1907-05-05 12:34:56.1279,1907-12-25 +1907-05-05 12:34:56.128,1907-12-25 +1907-05-05 12:34:56.1281,1907-12-25 +1907-05-05 12:34:56.1282,1907-12-25 +1907-05-05 12:34:56.1283,1907-12-25 +1907-05-05 12:34:56.1284,1907-12-25 +1907-05-05 12:34:56.1285,1907-12-25 +1907-05-05 12:34:56.1286,1907-12-25 +1907-05-05 12:34:56.1287,1907-12-25 +1907-05-05 12:34:56.1288,1907-12-25 +1907-05-05 12:34:56.1289,1907-12-25 +1907-05-05 12:34:56.129,1907-12-25 +1907-05-05 12:34:56.1291,1907-12-25 +1907-05-05 12:34:56.1292,1907-12-25 +1907-05-05 12:34:56.1293,1907-12-25 +1907-05-05 12:34:56.1294,1907-12-25 +1907-05-05 12:34:56.1295,1907-12-25 +1907-05-05 12:34:56.1296,1907-12-25 +1907-05-05 12:34:56.1297,1907-12-25 +1907-05-05 12:34:56.1298,1907-12-25 +1907-05-05 12:34:56.1299,1907-12-25 +1907-05-05 12:34:56.13,1907-12-25 +1907-05-05 12:34:56.1301,1907-12-25 +1907-05-05 12:34:56.1302,1907-12-25 +1907-05-05 12:34:56.1303,1907-12-25 +1907-05-05 12:34:56.1304,1907-12-25 +1907-05-05 12:34:56.1305,1907-12-25 +1907-05-05 12:34:56.1306,1907-12-25 +1907-05-05 12:34:56.1307,1907-12-25 +1907-05-05 12:34:56.1308,1907-12-25 +1907-05-05 12:34:56.1309,1907-12-25 +1907-05-05 12:34:56.131,1907-12-25 +1907-05-05 12:34:56.1311,1907-12-25 +1907-05-05 12:34:56.1312,1907-12-25 +1907-05-05 12:34:56.1313,1907-12-25 +1907-05-05 12:34:56.1314,1907-12-25 +1907-05-05 12:34:56.1315,1907-12-25 +1907-05-05 12:34:56.1316,1907-12-25 +1907-05-05 12:34:56.1317,1907-12-25 +1907-05-05 12:34:56.1318,1907-12-25 +1907-05-05 12:34:56.1319,1907-12-25 +1907-05-05 12:34:56.132,1907-12-25 +1907-05-05 12:34:56.1321,1907-12-25 +1907-05-05 12:34:56.1322,1907-12-25 +1907-05-05 12:34:56.1323,1907-12-25 +1907-05-05 12:34:56.1324,1907-12-25 +1907-05-05 12:34:56.1325,1907-12-25 +1907-05-05 12:34:56.1326,1907-12-25 +1907-05-05 12:34:56.1327,1907-12-25 +1907-05-05 12:34:56.1328,1907-12-25 +1907-05-05 12:34:56.1329,1907-12-25 +1907-05-05 12:34:56.133,1907-12-25 +1907-05-05 12:34:56.1331,1907-12-25 +1907-05-05 12:34:56.1332,1907-12-25 +1907-05-05 12:34:56.1333,1907-12-25 +1907-05-05 12:34:56.1334,1907-12-25 +1907-05-05 12:34:56.1335,1907-12-25 +1907-05-05 12:34:56.1336,1907-12-25 +1907-05-05 12:34:56.1337,1907-12-25 +1907-05-05 12:34:56.1338,1907-12-25 +1907-05-05 12:34:56.1339,1907-12-25 +1907-05-05 12:34:56.134,1907-12-25 +1907-05-05 12:34:56.1341,1907-12-25 +1907-05-05 12:34:56.1342,1907-12-25 +1907-05-05 12:34:56.1343,1907-12-25 +1907-05-05 12:34:56.1344,1907-12-25 +1907-05-05 12:34:56.1345,1907-12-25 +1907-05-05 12:34:56.1346,1907-12-25 +1907-05-05 12:34:56.1347,1907-12-25 +1907-05-05 12:34:56.1348,1907-12-25 +1907-05-05 12:34:56.1349,1907-12-25 +1907-05-05 12:34:56.135,1907-12-25 +1907-05-05 12:34:56.1351,1907-12-25 +1907-05-05 12:34:56.1352,1907-12-25 +1907-05-05 12:34:56.1353,1907-12-25 +1907-05-05 12:34:56.1354,1907-12-25 +1907-05-05 12:34:56.1355,1907-12-25 +1907-05-05 12:34:56.1356,1907-12-25 +1907-05-05 12:34:56.1357,1907-12-25 +1907-05-05 12:34:56.1358,1907-12-25 +1907-05-05 12:34:56.1359,1907-12-25 +1907-05-05 12:34:56.136,1907-12-25 +1907-05-05 12:34:56.1361,1907-12-25 +1907-05-05 12:34:56.1362,1907-12-25 +1907-05-05 12:34:56.1363,1907-12-25 +1907-05-05 12:34:56.1364,1907-12-25 +1907-05-05 12:34:56.1365,1907-12-25 +1907-05-05 12:34:56.1366,1907-12-25 +1907-05-05 12:34:56.1367,1907-12-25 +1907-05-05 12:34:56.1368,1907-12-25 +1907-05-05 12:34:56.1369,1907-12-25 +1907-05-05 12:34:56.137,1907-12-25 +1907-05-05 12:34:56.1371,1907-12-25 +1907-05-05 12:34:56.1372,1907-12-25 +1907-05-05 12:34:56.1373,1907-12-25 +1907-05-05 12:34:56.1374,1907-12-25 +1907-05-05 12:34:56.1375,1907-12-25 +1907-05-05 12:34:56.1376,1907-12-25 +1907-05-05 12:34:56.1377,1907-12-25 +1907-05-05 12:34:56.1378,1907-12-25 +1907-05-05 12:34:56.1379,1907-12-25 +1907-05-05 12:34:56.138,1907-12-25 +1907-05-05 12:34:56.1381,1907-12-25 +1907-05-05 12:34:56.1382,1907-12-25 +1907-05-05 12:34:56.1383,1907-12-25 +1907-05-05 12:34:56.1384,1907-12-25 +1907-05-05 12:34:56.1385,1907-12-25 +1907-05-05 12:34:56.1386,1907-12-25 +1907-05-05 12:34:56.1387,1907-12-25 +1907-05-05 12:34:56.1388,1907-12-25 +1907-05-05 12:34:56.1389,1907-12-25 +1907-05-05 12:34:56.139,1907-12-25 +1907-05-05 12:34:56.1391,1907-12-25 +1907-05-05 12:34:56.1392,1907-12-25 +1907-05-05 12:34:56.1393,1907-12-25 +1907-05-05 12:34:56.1394,1907-12-25 +1907-05-05 12:34:56.1395,1907-12-25 +1907-05-05 12:34:56.1396,1907-12-25 +1907-05-05 12:34:56.1397,1907-12-25 +1907-05-05 12:34:56.1398,1907-12-25 +1907-05-05 12:34:56.1399,1907-12-25 +1907-05-05 12:34:56.14,1907-12-25 +1907-05-05 12:34:56.1401,1907-12-25 +1907-05-05 12:34:56.1402,1907-12-25 +1907-05-05 12:34:56.1403,1907-12-25 +1907-05-05 12:34:56.1404,1907-12-25 +1907-05-05 12:34:56.1405,1907-12-25 +1907-05-05 12:34:56.1406,1907-12-25 +1907-05-05 12:34:56.1407,1907-12-25 +1907-05-05 12:34:56.1408,1907-12-25 +1907-05-05 12:34:56.1409,1907-12-25 +1907-05-05 12:34:56.141,1907-12-25 +1907-05-05 12:34:56.1411,1907-12-25 +1907-05-05 12:34:56.1412,1907-12-25 +1907-05-05 12:34:56.1413,1907-12-25 +1907-05-05 12:34:56.1414,1907-12-25 +1907-05-05 12:34:56.1415,1907-12-25 +1907-05-05 12:34:56.1416,1907-12-25 +1907-05-05 12:34:56.1417,1907-12-25 +1907-05-05 12:34:56.1418,1907-12-25 +1907-05-05 12:34:56.1419,1907-12-25 +1907-05-05 12:34:56.142,1907-12-25 +1907-05-05 12:34:56.1421,1907-12-25 +1907-05-05 12:34:56.1422,1907-12-25 +1907-05-05 12:34:56.1423,1907-12-25 +1907-05-05 12:34:56.1424,1907-12-25 +1907-05-05 12:34:56.1425,1907-12-25 +1907-05-05 12:34:56.1426,1907-12-25 +1907-05-05 12:34:56.1427,1907-12-25 +1907-05-05 12:34:56.1428,1907-12-25 +1907-05-05 12:34:56.1429,1907-12-25 +1907-05-05 12:34:56.143,1907-12-25 +1907-05-05 12:34:56.1431,1907-12-25 +1907-05-05 12:34:56.1432,1907-12-25 +1907-05-05 12:34:56.1433,1907-12-25 +1907-05-05 12:34:56.1434,1907-12-25 +1907-05-05 12:34:56.1435,1907-12-25 +1907-05-05 12:34:56.1436,1907-12-25 +1907-05-05 12:34:56.1437,1907-12-25 +1907-05-05 12:34:56.1438,1907-12-25 +1907-05-05 12:34:56.1439,1907-12-25 +1907-05-05 12:34:56.144,1907-12-25 +1907-05-05 12:34:56.1441,1907-12-25 +1907-05-05 12:34:56.1442,1907-12-25 +1907-05-05 12:34:56.1443,1907-12-25 +1907-05-05 12:34:56.1444,1907-12-25 +1907-05-05 12:34:56.1445,1907-12-25 +1907-05-05 12:34:56.1446,1907-12-25 +1907-05-05 12:34:56.1447,1907-12-25 +1907-05-05 12:34:56.1448,1907-12-25 +1907-05-05 12:34:56.1449,1907-12-25 +1907-05-05 12:34:56.145,1907-12-25 +1907-05-05 12:34:56.1451,1907-12-25 +1907-05-05 12:34:56.1452,1907-12-25 +1907-05-05 12:34:56.1453,1907-12-25 +1907-05-05 12:34:56.1454,1907-12-25 +1907-05-05 12:34:56.1455,1907-12-25 +1907-05-05 12:34:56.1456,1907-12-25 +1907-05-05 12:34:56.1457,1907-12-25 +1907-05-05 12:34:56.1458,1907-12-25 +1907-05-05 12:34:56.1459,1907-12-25 +1907-05-05 12:34:56.146,1907-12-25 +1907-05-05 12:34:56.1461,1907-12-25 +1907-05-05 12:34:56.1462,1907-12-25 +1907-05-05 12:34:56.1463,1907-12-25 +1907-05-05 12:34:56.1464,1907-12-25 +1907-05-05 12:34:56.1465,1907-12-25 +1907-05-05 12:34:56.1466,1907-12-25 +1907-05-05 12:34:56.1467,1907-12-25 +1907-05-05 12:34:56.1468,1907-12-25 +1907-05-05 12:34:56.1469,1907-12-25 +1907-05-05 12:34:56.147,1907-12-25 +1907-05-05 12:34:56.1471,1907-12-25 +1907-05-05 12:34:56.1472,1907-12-25 +1907-05-05 12:34:56.1473,1907-12-25 +1907-05-05 12:34:56.1474,1907-12-25 +1907-05-05 12:34:56.1475,1907-12-25 +1907-05-05 12:34:56.1476,1907-12-25 +1907-05-05 12:34:56.1477,1907-12-25 +1907-05-05 12:34:56.1478,1907-12-25 +1907-05-05 12:34:56.1479,1907-12-25 +1907-05-05 12:34:56.148,1907-12-25 +1907-05-05 12:34:56.1481,1907-12-25 +1907-05-05 12:34:56.1482,1907-12-25 +1907-05-05 12:34:56.1483,1907-12-25 +1907-05-05 12:34:56.1484,1907-12-25 +1907-05-05 12:34:56.1485,1907-12-25 +1907-05-05 12:34:56.1486,1907-12-25 +1907-05-05 12:34:56.1487,1907-12-25 +1907-05-05 12:34:56.1488,1907-12-25 +1907-05-05 12:34:56.1489,1907-12-25 +1907-05-05 12:34:56.149,1907-12-25 +1907-05-05 12:34:56.1491,1907-12-25 +1907-05-05 12:34:56.1492,1907-12-25 +1907-05-05 12:34:56.1493,1907-12-25 +1907-05-05 12:34:56.1494,1907-12-25 +1907-05-05 12:34:56.1495,1907-12-25 +1907-05-05 12:34:56.1496,1907-12-25 +1907-05-05 12:34:56.1497,1907-12-25 +1907-05-05 12:34:56.1498,1907-12-25 +1907-05-05 12:34:56.1499,1907-12-25 +1907-05-05 12:34:56.15,1907-12-25 +1907-05-05 12:34:56.1501,1907-12-25 +1907-05-05 12:34:56.1502,1907-12-25 +1907-05-05 12:34:56.1503,1907-12-25 +1907-05-05 12:34:56.1504,1907-12-25 +1907-05-05 12:34:56.1505,1907-12-25 +1907-05-05 12:34:56.1506,1907-12-25 +1907-05-05 12:34:56.1507,1907-12-25 +1907-05-05 12:34:56.1508,1907-12-25 +1907-05-05 12:34:56.1509,1907-12-25 +1907-05-05 12:34:56.151,1907-12-25 +1907-05-05 12:34:56.1511,1907-12-25 +1907-05-05 12:34:56.1512,1907-12-25 +1907-05-05 12:34:56.1513,1907-12-25 +1907-05-05 12:34:56.1514,1907-12-25 +1907-05-05 12:34:56.1515,1907-12-25 +1907-05-05 12:34:56.1516,1907-12-25 +1907-05-05 12:34:56.1517,1907-12-25 +1907-05-05 12:34:56.1518,1907-12-25 +1907-05-05 12:34:56.1519,1907-12-25 +1907-05-05 12:34:56.152,1907-12-25 +1907-05-05 12:34:56.1521,1907-12-25 +1907-05-05 12:34:56.1522,1907-12-25 +1907-05-05 12:34:56.1523,1907-12-25 +1907-05-05 12:34:56.1524,1907-12-25 +1907-05-05 12:34:56.1525,1907-12-25 +1907-05-05 12:34:56.1526,1907-12-25 +1907-05-05 12:34:56.1527,1907-12-25 +1907-05-05 12:34:56.1528,1907-12-25 +1907-05-05 12:34:56.1529,1907-12-25 +1907-05-05 12:34:56.153,1907-12-25 +1907-05-05 12:34:56.1531,1907-12-25 +1907-05-05 12:34:56.1532,1907-12-25 +1907-05-05 12:34:56.1533,1907-12-25 +1907-05-05 12:34:56.1534,1907-12-25 +1907-05-05 12:34:56.1535,1907-12-25 +1907-05-05 12:34:56.1536,1907-12-25 +1907-05-05 12:34:56.1537,1907-12-25 +1907-05-05 12:34:56.1538,1907-12-25 +1907-05-05 12:34:56.1539,1907-12-25 +1907-05-05 12:34:56.154,1907-12-25 +1907-05-05 12:34:56.1541,1907-12-25 +1907-05-05 12:34:56.1542,1907-12-25 +1907-05-05 12:34:56.1543,1907-12-25 +1907-05-05 12:34:56.1544,1907-12-25 +1907-05-05 12:34:56.1545,1907-12-25 +1907-05-05 12:34:56.1546,1907-12-25 +1907-05-05 12:34:56.1547,1907-12-25 +1907-05-05 12:34:56.1548,1907-12-25 +1907-05-05 12:34:56.1549,1907-12-25 +1907-05-05 12:34:56.155,1907-12-25 +1907-05-05 12:34:56.1551,1907-12-25 +1907-05-05 12:34:56.1552,1907-12-25 +1907-05-05 12:34:56.1553,1907-12-25 +1907-05-05 12:34:56.1554,1907-12-25 +1907-05-05 12:34:56.1555,1907-12-25 +1907-05-05 12:34:56.1556,1907-12-25 +1907-05-05 12:34:56.1557,1907-12-25 +1907-05-05 12:34:56.1558,1907-12-25 +1907-05-05 12:34:56.1559,1907-12-25 +1907-05-05 12:34:56.156,1907-12-25 +1907-05-05 12:34:56.1561,1907-12-25 +1907-05-05 12:34:56.1562,1907-12-25 +1907-05-05 12:34:56.1563,1907-12-25 +1907-05-05 12:34:56.1564,1907-12-25 +1907-05-05 12:34:56.1565,1907-12-25 +1907-05-05 12:34:56.1566,1907-12-25 +1907-05-05 12:34:56.1567,1907-12-25 +1907-05-05 12:34:56.1568,1907-12-25 +1907-05-05 12:34:56.1569,1907-12-25 +1907-05-05 12:34:56.157,1907-12-25 +1907-05-05 12:34:56.1571,1907-12-25 +1907-05-05 12:34:56.1572,1907-12-25 +1907-05-05 12:34:56.1573,1907-12-25 +1907-05-05 12:34:56.1574,1907-12-25 +1907-05-05 12:34:56.1575,1907-12-25 +1907-05-05 12:34:56.1576,1907-12-25 +1907-05-05 12:34:56.1577,1907-12-25 +1907-05-05 12:34:56.1578,1907-12-25 +1907-05-05 12:34:56.1579,1907-12-25 +1907-05-05 12:34:56.158,1907-12-25 +1907-05-05 12:34:56.1581,1907-12-25 +1907-05-05 12:34:56.1582,1907-12-25 +1907-05-05 12:34:56.1583,1907-12-25 +1907-05-05 12:34:56.1584,1907-12-25 +1907-05-05 12:34:56.1585,1907-12-25 +1907-05-05 12:34:56.1586,1907-12-25 +1907-05-05 12:34:56.1587,1907-12-25 +1907-05-05 12:34:56.1588,1907-12-25 +1907-05-05 12:34:56.1589,1907-12-25 +1907-05-05 12:34:56.159,1907-12-25 +1907-05-05 12:34:56.1591,1907-12-25 +1907-05-05 12:34:56.1592,1907-12-25 +1907-05-05 12:34:56.1593,1907-12-25 +1907-05-05 12:34:56.1594,1907-12-25 +1907-05-05 12:34:56.1595,1907-12-25 +1907-05-05 12:34:56.1596,1907-12-25 +1907-05-05 12:34:56.1597,1907-12-25 +1907-05-05 12:34:56.1598,1907-12-25 +1907-05-05 12:34:56.1599,1907-12-25 +1907-05-05 12:34:56.16,1907-12-25 +1907-05-05 12:34:56.1601,1907-12-25 +1907-05-05 12:34:56.1602,1907-12-25 +1907-05-05 12:34:56.1603,1907-12-25 +1907-05-05 12:34:56.1604,1907-12-25 +1907-05-05 12:34:56.1605,1907-12-25 +1907-05-05 12:34:56.1606,1907-12-25 +1907-05-05 12:34:56.1607,1907-12-25 +1907-05-05 12:34:56.1608,1907-12-25 +1907-05-05 12:34:56.1609,1907-12-25 +1907-05-05 12:34:56.161,1907-12-25 +1907-05-05 12:34:56.1611,1907-12-25 +1907-05-05 12:34:56.1612,1907-12-25 +1907-05-05 12:34:56.1613,1907-12-25 +1907-05-05 12:34:56.1614,1907-12-25 +1907-05-05 12:34:56.1615,1907-12-25 +1907-05-05 12:34:56.1616,1907-12-25 +1907-05-05 12:34:56.1617,1907-12-25 +1907-05-05 12:34:56.1618,1907-12-25 +1907-05-05 12:34:56.1619,1907-12-25 +1907-05-05 12:34:56.162,1907-12-25 +1907-05-05 12:34:56.1621,1907-12-25 +1907-05-05 12:34:56.1622,1907-12-25 +1907-05-05 12:34:56.1623,1907-12-25 +1907-05-05 12:34:56.1624,1907-12-25 +1907-05-05 12:34:56.1625,1907-12-25 +1907-05-05 12:34:56.1626,1907-12-25 +1907-05-05 12:34:56.1627,1907-12-25 +1907-05-05 12:34:56.1628,1907-12-25 +1907-05-05 12:34:56.1629,1907-12-25 +1907-05-05 12:34:56.163,1907-12-25 +1907-05-05 12:34:56.1631,1907-12-25 +1907-05-05 12:34:56.1632,1907-12-25 +1907-05-05 12:34:56.1633,1907-12-25 +1907-05-05 12:34:56.1634,1907-12-25 +1907-05-05 12:34:56.1635,1907-12-25 +1907-05-05 12:34:56.1636,1907-12-25 +1907-05-05 12:34:56.1637,1907-12-25 +1907-05-05 12:34:56.1638,1907-12-25 +1907-05-05 12:34:56.1639,1907-12-25 +1907-05-05 12:34:56.164,1907-12-25 +1907-05-05 12:34:56.1641,1907-12-25 +1907-05-05 12:34:56.1642,1907-12-25 +1907-05-05 12:34:56.1643,1907-12-25 +1907-05-05 12:34:56.1644,1907-12-25 +1907-05-05 12:34:56.1645,1907-12-25 +1907-05-05 12:34:56.1646,1907-12-25 +1907-05-05 12:34:56.1647,1907-12-25 +1907-05-05 12:34:56.1648,1907-12-25 +1907-05-05 12:34:56.1649,1907-12-25 +1907-05-05 12:34:56.165,1907-12-25 +1907-05-05 12:34:56.1651,1907-12-25 +1907-05-05 12:34:56.1652,1907-12-25 +1907-05-05 12:34:56.1653,1907-12-25 +1907-05-05 12:34:56.1654,1907-12-25 +1907-05-05 12:34:56.1655,1907-12-25 +1907-05-05 12:34:56.1656,1907-12-25 +1907-05-05 12:34:56.1657,1907-12-25 +1907-05-05 12:34:56.1658,1907-12-25 +1907-05-05 12:34:56.1659,1907-12-25 +1907-05-05 12:34:56.166,1907-12-25 +1907-05-05 12:34:56.1661,1907-12-25 +1907-05-05 12:34:56.1662,1907-12-25 +1907-05-05 12:34:56.1663,1907-12-25 +1907-05-05 12:34:56.1664,1907-12-25 +1907-05-05 12:34:56.1665,1907-12-25 +1907-05-05 12:34:56.1666,1907-12-25 +1907-05-05 12:34:56.1667,1907-12-25 +1907-05-05 12:34:56.1668,1907-12-25 +1907-05-05 12:34:56.1669,1907-12-25 +1907-05-05 12:34:56.167,1907-12-25 +1907-05-05 12:34:56.1671,1907-12-25 +1907-05-05 12:34:56.1672,1907-12-25 +1907-05-05 12:34:56.1673,1907-12-25 +1907-05-05 12:34:56.1674,1907-12-25 +1907-05-05 12:34:56.1675,1907-12-25 +1907-05-05 12:34:56.1676,1907-12-25 +1907-05-05 12:34:56.1677,1907-12-25 +1907-05-05 12:34:56.1678,1907-12-25 +1907-05-05 12:34:56.1679,1907-12-25 +1907-05-05 12:34:56.168,1907-12-25 +1907-05-05 12:34:56.1681,1907-12-25 +1907-05-05 12:34:56.1682,1907-12-25 +1907-05-05 12:34:56.1683,1907-12-25 +1907-05-05 12:34:56.1684,1907-12-25 +1907-05-05 12:34:56.1685,1907-12-25 +1907-05-05 12:34:56.1686,1907-12-25 +1907-05-05 12:34:56.1687,1907-12-25 +1907-05-05 12:34:56.1688,1907-12-25 +1907-05-05 12:34:56.1689,1907-12-25 +1907-05-05 12:34:56.169,1907-12-25 +1907-05-05 12:34:56.1691,1907-12-25 +1907-05-05 12:34:56.1692,1907-12-25 +1907-05-05 12:34:56.1693,1907-12-25 +1907-05-05 12:34:56.1694,1907-12-25 +1907-05-05 12:34:56.1695,1907-12-25 +1907-05-05 12:34:56.1696,1907-12-25 +1907-05-05 12:34:56.1697,1907-12-25 +1907-05-05 12:34:56.1698,1907-12-25 +1907-05-05 12:34:56.1699,1907-12-25 +1907-05-05 12:34:56.17,1907-12-25 +1907-05-05 12:34:56.1701,1907-12-25 +1907-05-05 12:34:56.1702,1907-12-25 +1907-05-05 12:34:56.1703,1907-12-25 +1907-05-05 12:34:56.1704,1907-12-25 +1907-05-05 12:34:56.1705,1907-12-25 +1907-05-05 12:34:56.1706,1907-12-25 +1907-05-05 12:34:56.1707,1907-12-25 +1907-05-05 12:34:56.1708,1907-12-25 +1907-05-05 12:34:56.1709,1907-12-25 +1907-05-05 12:34:56.171,1907-12-25 +1907-05-05 12:34:56.1711,1907-12-25 +1907-05-05 12:34:56.1712,1907-12-25 +1907-05-05 12:34:56.1713,1907-12-25 +1907-05-05 12:34:56.1714,1907-12-25 +1907-05-05 12:34:56.1715,1907-12-25 +1907-05-05 12:34:56.1716,1907-12-25 +1907-05-05 12:34:56.1717,1907-12-25 +1907-05-05 12:34:56.1718,1907-12-25 +1907-05-05 12:34:56.1719,1907-12-25 +1907-05-05 12:34:56.172,1907-12-25 +1907-05-05 12:34:56.1721,1907-12-25 +1907-05-05 12:34:56.1722,1907-12-25 +1907-05-05 12:34:56.1723,1907-12-25 +1907-05-05 12:34:56.1724,1907-12-25 +1907-05-05 12:34:56.1725,1907-12-25 +1907-05-05 12:34:56.1726,1907-12-25 +1907-05-05 12:34:56.1727,1907-12-25 +1907-05-05 12:34:56.1728,1907-12-25 +1907-05-05 12:34:56.1729,1907-12-25 +1907-05-05 12:34:56.173,1907-12-25 +1907-05-05 12:34:56.1731,1907-12-25 +1907-05-05 12:34:56.1732,1907-12-25 +1907-05-05 12:34:56.1733,1907-12-25 +1907-05-05 12:34:56.1734,1907-12-25 +1907-05-05 12:34:56.1735,1907-12-25 +1907-05-05 12:34:56.1736,1907-12-25 +1907-05-05 12:34:56.1737,1907-12-25 +1907-05-05 12:34:56.1738,1907-12-25 +1907-05-05 12:34:56.1739,1907-12-25 +1907-05-05 12:34:56.174,1907-12-25 +1907-05-05 12:34:56.1741,1907-12-25 +1907-05-05 12:34:56.1742,1907-12-25 +1907-05-05 12:34:56.1743,1907-12-25 +1907-05-05 12:34:56.1744,1907-12-25 +1907-05-05 12:34:56.1745,1907-12-25 +1907-05-05 12:34:56.1746,1907-12-25 +1907-05-05 12:34:56.1747,1907-12-25 +1907-05-05 12:34:56.1748,1907-12-25 +1907-05-05 12:34:56.1749,1907-12-25 +1907-05-05 12:34:56.175,1907-12-25 +1907-05-05 12:34:56.1751,1907-12-25 +1907-05-05 12:34:56.1752,1907-12-25 +1907-05-05 12:34:56.1753,1907-12-25 +1907-05-05 12:34:56.1754,1907-12-25 +1907-05-05 12:34:56.1755,1907-12-25 +1907-05-05 12:34:56.1756,1907-12-25 +1907-05-05 12:34:56.1757,1907-12-25 +1907-05-05 12:34:56.1758,1907-12-25 +1907-05-05 12:34:56.1759,1907-12-25 +1907-05-05 12:34:56.176,1907-12-25 +1907-05-05 12:34:56.1761,1907-12-25 +1907-05-05 12:34:56.1762,1907-12-25 +1907-05-05 12:34:56.1763,1907-12-25 +1907-05-05 12:34:56.1764,1907-12-25 +1907-05-05 12:34:56.1765,1907-12-25 +1907-05-05 12:34:56.1766,1907-12-25 +1907-05-05 12:34:56.1767,1907-12-25 +1907-05-05 12:34:56.1768,1907-12-25 +1907-05-05 12:34:56.1769,1907-12-25 +1907-05-05 12:34:56.177,1907-12-25 +1907-05-05 12:34:56.1771,1907-12-25 +1907-05-05 12:34:56.1772,1907-12-25 +1907-05-05 12:34:56.1773,1907-12-25 +1907-05-05 12:34:56.1774,1907-12-25 +1907-05-05 12:34:56.1775,1907-12-25 +1907-05-05 12:34:56.1776,1907-12-25 +1907-05-05 12:34:56.1777,1907-12-25 +1907-05-05 12:34:56.1778,1907-12-25 +1907-05-05 12:34:56.1779,1907-12-25 +1907-05-05 12:34:56.178,1907-12-25 +1907-05-05 12:34:56.1781,1907-12-25 +1907-05-05 12:34:56.1782,1907-12-25 +1907-05-05 12:34:56.1783,1907-12-25 +1907-05-05 12:34:56.1784,1907-12-25 +1907-05-05 12:34:56.1785,1907-12-25 +1907-05-05 12:34:56.1786,1907-12-25 +1907-05-05 12:34:56.1787,1907-12-25 +1907-05-05 12:34:56.1788,1907-12-25 +1907-05-05 12:34:56.1789,1907-12-25 +1907-05-05 12:34:56.179,1907-12-25 +1907-05-05 12:34:56.1791,1907-12-25 +1907-05-05 12:34:56.1792,1907-12-25 +1907-05-05 12:34:56.1793,1907-12-25 +1907-05-05 12:34:56.1794,1907-12-25 +1907-05-05 12:34:56.1795,1907-12-25 +1907-05-05 12:34:56.1796,1907-12-25 +1907-05-05 12:34:56.1797,1907-12-25 +1907-05-05 12:34:56.1798,1907-12-25 +1907-05-05 12:34:56.1799,1907-12-25 +1907-05-05 12:34:56.18,1907-12-25 +1907-05-05 12:34:56.1801,1907-12-25 +1907-05-05 12:34:56.1802,1907-12-25 +1907-05-05 12:34:56.1803,1907-12-25 +1907-05-05 12:34:56.1804,1907-12-25 +1907-05-05 12:34:56.1805,1907-12-25 +1907-05-05 12:34:56.1806,1907-12-25 +1907-05-05 12:34:56.1807,1907-12-25 +1907-05-05 12:34:56.1808,1907-12-25 +1907-05-05 12:34:56.1809,1907-12-25 +1907-05-05 12:34:56.181,1907-12-25 +1907-05-05 12:34:56.1811,1907-12-25 +1907-05-05 12:34:56.1812,1907-12-25 +1907-05-05 12:34:56.1813,1907-12-25 +1907-05-05 12:34:56.1814,1907-12-25 +1907-05-05 12:34:56.1815,1907-12-25 +1907-05-05 12:34:56.1816,1907-12-25 +1907-05-05 12:34:56.1817,1907-12-25 +1907-05-05 12:34:56.1818,1907-12-25 +1907-05-05 12:34:56.1819,1907-12-25 +1907-05-05 12:34:56.182,1907-12-25 +1907-05-05 12:34:56.1821,1907-12-25 +1907-05-05 12:34:56.1822,1907-12-25 +1907-05-05 12:34:56.1823,1907-12-25 +1907-05-05 12:34:56.1824,1907-12-25 +1907-05-05 12:34:56.1825,1907-12-25 +1907-05-05 12:34:56.1826,1907-12-25 +1907-05-05 12:34:56.1827,1907-12-25 +1907-05-05 12:34:56.1828,1907-12-25 +1907-05-05 12:34:56.1829,1907-12-25 +1907-05-05 12:34:56.183,1907-12-25 +1907-05-05 12:34:56.1831,1907-12-25 +1907-05-05 12:34:56.1832,1907-12-25 +1907-05-05 12:34:56.1833,1907-12-25 +1907-05-05 12:34:56.1834,1907-12-25 +1907-05-05 12:34:56.1835,1907-12-25 +1907-05-05 12:34:56.1836,1907-12-25 +1907-05-05 12:34:56.1837,1907-12-25 +1907-05-05 12:34:56.1838,1907-12-25 +1907-05-05 12:34:56.1839,1907-12-25 +1907-05-05 12:34:56.184,1907-12-25 +1907-05-05 12:34:56.1841,1907-12-25 +1907-05-05 12:34:56.1842,1907-12-25 +1907-05-05 12:34:56.1843,1907-12-25 +1907-05-05 12:34:56.1844,1907-12-25 +1907-05-05 12:34:56.1845,1907-12-25 +1907-05-05 12:34:56.1846,1907-12-25 +1907-05-05 12:34:56.1847,1907-12-25 +1907-05-05 12:34:56.1848,1907-12-25 +1907-05-05 12:34:56.1849,1907-12-25 +1907-05-05 12:34:56.185,1907-12-25 +1907-05-05 12:34:56.1851,1907-12-25 +1907-05-05 12:34:56.1852,1907-12-25 +1907-05-05 12:34:56.1853,1907-12-25 +1907-05-05 12:34:56.1854,1907-12-25 +1907-05-05 12:34:56.1855,1907-12-25 +1907-05-05 12:34:56.1856,1907-12-25 +1907-05-05 12:34:56.1857,1907-12-25 +1907-05-05 12:34:56.1858,1907-12-25 +1907-05-05 12:34:56.1859,1907-12-25 +1907-05-05 12:34:56.186,1907-12-25 +1907-05-05 12:34:56.1861,1907-12-25 +1907-05-05 12:34:56.1862,1907-12-25 +1907-05-05 12:34:56.1863,1907-12-25 +1907-05-05 12:34:56.1864,1907-12-25 +1907-05-05 12:34:56.1865,1907-12-25 +1907-05-05 12:34:56.1866,1907-12-25 +1907-05-05 12:34:56.1867,1907-12-25 +1907-05-05 12:34:56.1868,1907-12-25 +1907-05-05 12:34:56.1869,1907-12-25 +1907-05-05 12:34:56.187,1907-12-25 +1907-05-05 12:34:56.1871,1907-12-25 +1907-05-05 12:34:56.1872,1907-12-25 +1907-05-05 12:34:56.1873,1907-12-25 +1907-05-05 12:34:56.1874,1907-12-25 +1907-05-05 12:34:56.1875,1907-12-25 +1907-05-05 12:34:56.1876,1907-12-25 +1907-05-05 12:34:56.1877,1907-12-25 +1907-05-05 12:34:56.1878,1907-12-25 +1907-05-05 12:34:56.1879,1907-12-25 +1907-05-05 12:34:56.188,1907-12-25 +1907-05-05 12:34:56.1881,1907-12-25 +1907-05-05 12:34:56.1882,1907-12-25 +1907-05-05 12:34:56.1883,1907-12-25 +1907-05-05 12:34:56.1884,1907-12-25 +1907-05-05 12:34:56.1885,1907-12-25 +1907-05-05 12:34:56.1886,1907-12-25 +1907-05-05 12:34:56.1887,1907-12-25 +1907-05-05 12:34:56.1888,1907-12-25 +1907-05-05 12:34:56.1889,1907-12-25 +1907-05-05 12:34:56.189,1907-12-25 +1907-05-05 12:34:56.1891,1907-12-25 +1907-05-05 12:34:56.1892,1907-12-25 +1907-05-05 12:34:56.1893,1907-12-25 +1907-05-05 12:34:56.1894,1907-12-25 +1907-05-05 12:34:56.1895,1907-12-25 +1907-05-05 12:34:56.1896,1907-12-25 +1907-05-05 12:34:56.1897,1907-12-25 +1907-05-05 12:34:56.1898,1907-12-25 +1907-05-05 12:34:56.1899,1907-12-25 +1907-05-05 12:34:56.19,1907-12-25 +1907-05-05 12:34:56.1901,1907-12-25 +1907-05-05 12:34:56.1902,1907-12-25 +1907-05-05 12:34:56.1903,1907-12-25 +1907-05-05 12:34:56.1904,1907-12-25 +1907-05-05 12:34:56.1905,1907-12-25 +1907-05-05 12:34:56.1906,1907-12-25 +1907-05-05 12:34:56.1907,1907-12-25 +1907-05-05 12:34:56.1908,1907-12-25 +1907-05-05 12:34:56.1909,1907-12-25 +1907-05-05 12:34:56.191,1907-12-25 +1907-05-05 12:34:56.1911,1907-12-25 +1907-05-05 12:34:56.1912,1907-12-25 +1907-05-05 12:34:56.1913,1907-12-25 +1907-05-05 12:34:56.1914,1907-12-25 +1907-05-05 12:34:56.1915,1907-12-25 +1907-05-05 12:34:56.1916,1907-12-25 +1907-05-05 12:34:56.1917,1907-12-25 +1907-05-05 12:34:56.1918,1907-12-25 +1907-05-05 12:34:56.1919,1907-12-25 +1907-05-05 12:34:56.192,1907-12-25 +1907-05-05 12:34:56.1921,1907-12-25 +1907-05-05 12:34:56.1922,1907-12-25 +1907-05-05 12:34:56.1923,1907-12-25 +1907-05-05 12:34:56.1924,1907-12-25 +1907-05-05 12:34:56.1925,1907-12-25 +1907-05-05 12:34:56.1926,1907-12-25 +1907-05-05 12:34:56.1927,1907-12-25 +1907-05-05 12:34:56.1928,1907-12-25 +1907-05-05 12:34:56.1929,1907-12-25 +1907-05-05 12:34:56.193,1907-12-25 +1907-05-05 12:34:56.1931,1907-12-25 +1907-05-05 12:34:56.1932,1907-12-25 +1907-05-05 12:34:56.1933,1907-12-25 +1907-05-05 12:34:56.1934,1907-12-25 +1907-05-05 12:34:56.1935,1907-12-25 +1907-05-05 12:34:56.1936,1907-12-25 +1907-05-05 12:34:56.1937,1907-12-25 +1907-05-05 12:34:56.1938,1907-12-25 +1907-05-05 12:34:56.1939,1907-12-25 +1907-05-05 12:34:56.194,1907-12-25 +1907-05-05 12:34:56.1941,1907-12-25 +1907-05-05 12:34:56.1942,1907-12-25 +1907-05-05 12:34:56.1943,1907-12-25 +1907-05-05 12:34:56.1944,1907-12-25 +1907-05-05 12:34:56.1945,1907-12-25 +1907-05-05 12:34:56.1946,1907-12-25 +1907-05-05 12:34:56.1947,1907-12-25 +1907-05-05 12:34:56.1948,1907-12-25 +1907-05-05 12:34:56.1949,1907-12-25 +1907-05-05 12:34:56.195,1907-12-25 +1907-05-05 12:34:56.1951,1907-12-25 +1907-05-05 12:34:56.1952,1907-12-25 +1907-05-05 12:34:56.1953,1907-12-25 +1907-05-05 12:34:56.1954,1907-12-25 +1907-05-05 12:34:56.1955,1907-12-25 +1907-05-05 12:34:56.1956,1907-12-25 +1907-05-05 12:34:56.1957,1907-12-25 +1907-05-05 12:34:56.1958,1907-12-25 +1907-05-05 12:34:56.1959,1907-12-25 +1907-05-05 12:34:56.196,1907-12-25 +1907-05-05 12:34:56.1961,1907-12-25 +1907-05-05 12:34:56.1962,1907-12-25 +1907-05-05 12:34:56.1963,1907-12-25 +1907-05-05 12:34:56.1964,1907-12-25 +1907-05-05 12:34:56.1965,1907-12-25 +1907-05-05 12:34:56.1966,1907-12-25 +1907-05-05 12:34:56.1967,1907-12-25 +1907-05-05 12:34:56.1968,1907-12-25 +1907-05-05 12:34:56.1969,1907-12-25 +1907-05-05 12:34:56.197,1907-12-25 +1907-05-05 12:34:56.1971,1907-12-25 +1907-05-05 12:34:56.1972,1907-12-25 +1907-05-05 12:34:56.1973,1907-12-25 +1907-05-05 12:34:56.1974,1907-12-25 +1907-05-05 12:34:56.1975,1907-12-25 +1907-05-05 12:34:56.1976,1907-12-25 +1907-05-05 12:34:56.1977,1907-12-25 +1907-05-05 12:34:56.1978,1907-12-25 +1907-05-05 12:34:56.1979,1907-12-25 +1907-05-05 12:34:56.198,1907-12-25 +1907-05-05 12:34:56.1981,1907-12-25 +1907-05-05 12:34:56.1982,1907-12-25 +1907-05-05 12:34:56.1983,1907-12-25 +1907-05-05 12:34:56.1984,1907-12-25 +1907-05-05 12:34:56.1985,1907-12-25 +1907-05-05 12:34:56.1986,1907-12-25 +1907-05-05 12:34:56.1987,1907-12-25 +1907-05-05 12:34:56.1988,1907-12-25 +1907-05-05 12:34:56.1989,1907-12-25 +1907-05-05 12:34:56.199,1907-12-25 +1907-05-05 12:34:56.1991,1907-12-25 +1907-05-05 12:34:56.1992,1907-12-25 +1907-05-05 12:34:56.1993,1907-12-25 +1907-05-05 12:34:56.1994,1907-12-25 +1907-05-05 12:34:56.1995,1907-12-25 +1907-05-05 12:34:56.1996,1907-12-25 +1907-05-05 12:34:56.1997,1907-12-25 +1907-05-05 12:34:56.1998,1907-12-25 +1907-05-05 12:34:56.1999,1907-12-25 +1908-05-05 12:34:56.1,1908-12-25 +1908-05-05 12:34:56.1001,1908-12-25 +1908-05-05 12:34:56.1002,1908-12-25 +1908-05-05 12:34:56.1003,1908-12-25 +1908-05-05 12:34:56.1004,1908-12-25 +1908-05-05 12:34:56.1005,1908-12-25 +1908-05-05 12:34:56.1006,1908-12-25 +1908-05-05 12:34:56.1007,1908-12-25 +1908-05-05 12:34:56.1008,1908-12-25 +1908-05-05 12:34:56.1009,1908-12-25 +1908-05-05 12:34:56.101,1908-12-25 +1908-05-05 12:34:56.1011,1908-12-25 +1908-05-05 12:34:56.1012,1908-12-25 +1908-05-05 12:34:56.1013,1908-12-25 +1908-05-05 12:34:56.1014,1908-12-25 +1908-05-05 12:34:56.1015,1908-12-25 +1908-05-05 12:34:56.1016,1908-12-25 +1908-05-05 12:34:56.1017,1908-12-25 +1908-05-05 12:34:56.1018,1908-12-25 +1908-05-05 12:34:56.1019,1908-12-25 +1908-05-05 12:34:56.102,1908-12-25 +1908-05-05 12:34:56.1021,1908-12-25 +1908-05-05 12:34:56.1022,1908-12-25 +1908-05-05 12:34:56.1023,1908-12-25 +1908-05-05 12:34:56.1024,1908-12-25 +1908-05-05 12:34:56.1025,1908-12-25 +1908-05-05 12:34:56.1026,1908-12-25 +1908-05-05 12:34:56.1027,1908-12-25 +1908-05-05 12:34:56.1028,1908-12-25 +1908-05-05 12:34:56.1029,1908-12-25 +1908-05-05 12:34:56.103,1908-12-25 +1908-05-05 12:34:56.1031,1908-12-25 +1908-05-05 12:34:56.1032,1908-12-25 +1908-05-05 12:34:56.1033,1908-12-25 +1908-05-05 12:34:56.1034,1908-12-25 +1908-05-05 12:34:56.1035,1908-12-25 +1908-05-05 12:34:56.1036,1908-12-25 +1908-05-05 12:34:56.1037,1908-12-25 +1908-05-05 12:34:56.1038,1908-12-25 +1908-05-05 12:34:56.1039,1908-12-25 +1908-05-05 12:34:56.104,1908-12-25 +1908-05-05 12:34:56.1041,1908-12-25 +1908-05-05 12:34:56.1042,1908-12-25 +1908-05-05 12:34:56.1043,1908-12-25 +1908-05-05 12:34:56.1044,1908-12-25 +1908-05-05 12:34:56.1045,1908-12-25 +1908-05-05 12:34:56.1046,1908-12-25 +1908-05-05 12:34:56.1047,1908-12-25 +1908-05-05 12:34:56.1048,1908-12-25 +1908-05-05 12:34:56.1049,1908-12-25 +1908-05-05 12:34:56.105,1908-12-25 +1908-05-05 12:34:56.1051,1908-12-25 +1908-05-05 12:34:56.1052,1908-12-25 +1908-05-05 12:34:56.1053,1908-12-25 +1908-05-05 12:34:56.1054,1908-12-25 +1908-05-05 12:34:56.1055,1908-12-25 +1908-05-05 12:34:56.1056,1908-12-25 +1908-05-05 12:34:56.1057,1908-12-25 +1908-05-05 12:34:56.1058,1908-12-25 +1908-05-05 12:34:56.1059,1908-12-25 +1908-05-05 12:34:56.106,1908-12-25 +1908-05-05 12:34:56.1061,1908-12-25 +1908-05-05 12:34:56.1062,1908-12-25 +1908-05-05 12:34:56.1063,1908-12-25 +1908-05-05 12:34:56.1064,1908-12-25 +1908-05-05 12:34:56.1065,1908-12-25 +1908-05-05 12:34:56.1066,1908-12-25 +1908-05-05 12:34:56.1067,1908-12-25 +1908-05-05 12:34:56.1068,1908-12-25 +1908-05-05 12:34:56.1069,1908-12-25 +1908-05-05 12:34:56.107,1908-12-25 +1908-05-05 12:34:56.1071,1908-12-25 +1908-05-05 12:34:56.1072,1908-12-25 +1908-05-05 12:34:56.1073,1908-12-25 +1908-05-05 12:34:56.1074,1908-12-25 +1908-05-05 12:34:56.1075,1908-12-25 +1908-05-05 12:34:56.1076,1908-12-25 +1908-05-05 12:34:56.1077,1908-12-25 +1908-05-05 12:34:56.1078,1908-12-25 +1908-05-05 12:34:56.1079,1908-12-25 +1908-05-05 12:34:56.108,1908-12-25 +1908-05-05 12:34:56.1081,1908-12-25 +1908-05-05 12:34:56.1082,1908-12-25 +1908-05-05 12:34:56.1083,1908-12-25 +1908-05-05 12:34:56.1084,1908-12-25 +1908-05-05 12:34:56.1085,1908-12-25 +1908-05-05 12:34:56.1086,1908-12-25 +1908-05-05 12:34:56.1087,1908-12-25 +1908-05-05 12:34:56.1088,1908-12-25 +1908-05-05 12:34:56.1089,1908-12-25 +1908-05-05 12:34:56.109,1908-12-25 +1908-05-05 12:34:56.1091,1908-12-25 +1908-05-05 12:34:56.1092,1908-12-25 +1908-05-05 12:34:56.1093,1908-12-25 +1908-05-05 12:34:56.1094,1908-12-25 +1908-05-05 12:34:56.1095,1908-12-25 +1908-05-05 12:34:56.1096,1908-12-25 +1908-05-05 12:34:56.1097,1908-12-25 +1908-05-05 12:34:56.1098,1908-12-25 +1908-05-05 12:34:56.1099,1908-12-25 +1908-05-05 12:34:56.11,1908-12-25 +1908-05-05 12:34:56.1101,1908-12-25 +1908-05-05 12:34:56.1102,1908-12-25 +1908-05-05 12:34:56.1103,1908-12-25 +1908-05-05 12:34:56.1104,1908-12-25 +1908-05-05 12:34:56.1105,1908-12-25 +1908-05-05 12:34:56.1106,1908-12-25 +1908-05-05 12:34:56.1107,1908-12-25 +1908-05-05 12:34:56.1108,1908-12-25 +1908-05-05 12:34:56.1109,1908-12-25 +1908-05-05 12:34:56.111,1908-12-25 +1908-05-05 12:34:56.1111,1908-12-25 +1908-05-05 12:34:56.1112,1908-12-25 +1908-05-05 12:34:56.1113,1908-12-25 +1908-05-05 12:34:56.1114,1908-12-25 +1908-05-05 12:34:56.1115,1908-12-25 +1908-05-05 12:34:56.1116,1908-12-25 +1908-05-05 12:34:56.1117,1908-12-25 +1908-05-05 12:34:56.1118,1908-12-25 +1908-05-05 12:34:56.1119,1908-12-25 +1908-05-05 12:34:56.112,1908-12-25 +1908-05-05 12:34:56.1121,1908-12-25 +1908-05-05 12:34:56.1122,1908-12-25 +1908-05-05 12:34:56.1123,1908-12-25 +1908-05-05 12:34:56.1124,1908-12-25 +1908-05-05 12:34:56.1125,1908-12-25 +1908-05-05 12:34:56.1126,1908-12-25 +1908-05-05 12:34:56.1127,1908-12-25 +1908-05-05 12:34:56.1128,1908-12-25 +1908-05-05 12:34:56.1129,1908-12-25 +1908-05-05 12:34:56.113,1908-12-25 +1908-05-05 12:34:56.1131,1908-12-25 +1908-05-05 12:34:56.1132,1908-12-25 +1908-05-05 12:34:56.1133,1908-12-25 +1908-05-05 12:34:56.1134,1908-12-25 +1908-05-05 12:34:56.1135,1908-12-25 +1908-05-05 12:34:56.1136,1908-12-25 +1908-05-05 12:34:56.1137,1908-12-25 +1908-05-05 12:34:56.1138,1908-12-25 +1908-05-05 12:34:56.1139,1908-12-25 +1908-05-05 12:34:56.114,1908-12-25 +1908-05-05 12:34:56.1141,1908-12-25 +1908-05-05 12:34:56.1142,1908-12-25 +1908-05-05 12:34:56.1143,1908-12-25 +1908-05-05 12:34:56.1144,1908-12-25 +1908-05-05 12:34:56.1145,1908-12-25 +1908-05-05 12:34:56.1146,1908-12-25 +1908-05-05 12:34:56.1147,1908-12-25 +1908-05-05 12:34:56.1148,1908-12-25 +1908-05-05 12:34:56.1149,1908-12-25 +1908-05-05 12:34:56.115,1908-12-25 +1908-05-05 12:34:56.1151,1908-12-25 +1908-05-05 12:34:56.1152,1908-12-25 +1908-05-05 12:34:56.1153,1908-12-25 +1908-05-05 12:34:56.1154,1908-12-25 +1908-05-05 12:34:56.1155,1908-12-25 +1908-05-05 12:34:56.1156,1908-12-25 +1908-05-05 12:34:56.1157,1908-12-25 +1908-05-05 12:34:56.1158,1908-12-25 +1908-05-05 12:34:56.1159,1908-12-25 +1908-05-05 12:34:56.116,1908-12-25 +1908-05-05 12:34:56.1161,1908-12-25 +1908-05-05 12:34:56.1162,1908-12-25 +1908-05-05 12:34:56.1163,1908-12-25 +1908-05-05 12:34:56.1164,1908-12-25 +1908-05-05 12:34:56.1165,1908-12-25 +1908-05-05 12:34:56.1166,1908-12-25 +1908-05-05 12:34:56.1167,1908-12-25 +1908-05-05 12:34:56.1168,1908-12-25 +1908-05-05 12:34:56.1169,1908-12-25 +1908-05-05 12:34:56.117,1908-12-25 +1908-05-05 12:34:56.1171,1908-12-25 +1908-05-05 12:34:56.1172,1908-12-25 +1908-05-05 12:34:56.1173,1908-12-25 +1908-05-05 12:34:56.1174,1908-12-25 +1908-05-05 12:34:56.1175,1908-12-25 +1908-05-05 12:34:56.1176,1908-12-25 +1908-05-05 12:34:56.1177,1908-12-25 +1908-05-05 12:34:56.1178,1908-12-25 +1908-05-05 12:34:56.1179,1908-12-25 +1908-05-05 12:34:56.118,1908-12-25 +1908-05-05 12:34:56.1181,1908-12-25 +1908-05-05 12:34:56.1182,1908-12-25 +1908-05-05 12:34:56.1183,1908-12-25 +1908-05-05 12:34:56.1184,1908-12-25 +1908-05-05 12:34:56.1185,1908-12-25 +1908-05-05 12:34:56.1186,1908-12-25 +1908-05-05 12:34:56.1187,1908-12-25 +1908-05-05 12:34:56.1188,1908-12-25 +1908-05-05 12:34:56.1189,1908-12-25 +1908-05-05 12:34:56.119,1908-12-25 +1908-05-05 12:34:56.1191,1908-12-25 +1908-05-05 12:34:56.1192,1908-12-25 +1908-05-05 12:34:56.1193,1908-12-25 +1908-05-05 12:34:56.1194,1908-12-25 +1908-05-05 12:34:56.1195,1908-12-25 +1908-05-05 12:34:56.1196,1908-12-25 +1908-05-05 12:34:56.1197,1908-12-25 +1908-05-05 12:34:56.1198,1908-12-25 +1908-05-05 12:34:56.1199,1908-12-25 +1908-05-05 12:34:56.12,1908-12-25 +1908-05-05 12:34:56.1201,1908-12-25 +1908-05-05 12:34:56.1202,1908-12-25 +1908-05-05 12:34:56.1203,1908-12-25 +1908-05-05 12:34:56.1204,1908-12-25 +1908-05-05 12:34:56.1205,1908-12-25 +1908-05-05 12:34:56.1206,1908-12-25 +1908-05-05 12:34:56.1207,1908-12-25 +1908-05-05 12:34:56.1208,1908-12-25 +1908-05-05 12:34:56.1209,1908-12-25 +1908-05-05 12:34:56.121,1908-12-25 +1908-05-05 12:34:56.1211,1908-12-25 +1908-05-05 12:34:56.1212,1908-12-25 +1908-05-05 12:34:56.1213,1908-12-25 +1908-05-05 12:34:56.1214,1908-12-25 +1908-05-05 12:34:56.1215,1908-12-25 +1908-05-05 12:34:56.1216,1908-12-25 +1908-05-05 12:34:56.1217,1908-12-25 +1908-05-05 12:34:56.1218,1908-12-25 +1908-05-05 12:34:56.1219,1908-12-25 +1908-05-05 12:34:56.122,1908-12-25 +1908-05-05 12:34:56.1221,1908-12-25 +1908-05-05 12:34:56.1222,1908-12-25 +1908-05-05 12:34:56.1223,1908-12-25 +1908-05-05 12:34:56.1224,1908-12-25 +1908-05-05 12:34:56.1225,1908-12-25 +1908-05-05 12:34:56.1226,1908-12-25 +1908-05-05 12:34:56.1227,1908-12-25 +1908-05-05 12:34:56.1228,1908-12-25 +1908-05-05 12:34:56.1229,1908-12-25 +1908-05-05 12:34:56.123,1908-12-25 +1908-05-05 12:34:56.1231,1908-12-25 +1908-05-05 12:34:56.1232,1908-12-25 +1908-05-05 12:34:56.1233,1908-12-25 +1908-05-05 12:34:56.1234,1908-12-25 +1908-05-05 12:34:56.1235,1908-12-25 +1908-05-05 12:34:56.1236,1908-12-25 +1908-05-05 12:34:56.1237,1908-12-25 +1908-05-05 12:34:56.1238,1908-12-25 +1908-05-05 12:34:56.1239,1908-12-25 +1908-05-05 12:34:56.124,1908-12-25 +1908-05-05 12:34:56.1241,1908-12-25 +1908-05-05 12:34:56.1242,1908-12-25 +1908-05-05 12:34:56.1243,1908-12-25 +1908-05-05 12:34:56.1244,1908-12-25 +1908-05-05 12:34:56.1245,1908-12-25 +1908-05-05 12:34:56.1246,1908-12-25 +1908-05-05 12:34:56.1247,1908-12-25 +1908-05-05 12:34:56.1248,1908-12-25 +1908-05-05 12:34:56.1249,1908-12-25 +1908-05-05 12:34:56.125,1908-12-25 +1908-05-05 12:34:56.1251,1908-12-25 +1908-05-05 12:34:56.1252,1908-12-25 +1908-05-05 12:34:56.1253,1908-12-25 +1908-05-05 12:34:56.1254,1908-12-25 +1908-05-05 12:34:56.1255,1908-12-25 +1908-05-05 12:34:56.1256,1908-12-25 +1908-05-05 12:34:56.1257,1908-12-25 +1908-05-05 12:34:56.1258,1908-12-25 +1908-05-05 12:34:56.1259,1908-12-25 +1908-05-05 12:34:56.126,1908-12-25 +1908-05-05 12:34:56.1261,1908-12-25 +1908-05-05 12:34:56.1262,1908-12-25 +1908-05-05 12:34:56.1263,1908-12-25 +1908-05-05 12:34:56.1264,1908-12-25 +1908-05-05 12:34:56.1265,1908-12-25 +1908-05-05 12:34:56.1266,1908-12-25 +1908-05-05 12:34:56.1267,1908-12-25 +1908-05-05 12:34:56.1268,1908-12-25 +1908-05-05 12:34:56.1269,1908-12-25 +1908-05-05 12:34:56.127,1908-12-25 +1908-05-05 12:34:56.1271,1908-12-25 +1908-05-05 12:34:56.1272,1908-12-25 +1908-05-05 12:34:56.1273,1908-12-25 +1908-05-05 12:34:56.1274,1908-12-25 +1908-05-05 12:34:56.1275,1908-12-25 +1908-05-05 12:34:56.1276,1908-12-25 +1908-05-05 12:34:56.1277,1908-12-25 +1908-05-05 12:34:56.1278,1908-12-25 +1908-05-05 12:34:56.1279,1908-12-25 +1908-05-05 12:34:56.128,1908-12-25 +1908-05-05 12:34:56.1281,1908-12-25 +1908-05-05 12:34:56.1282,1908-12-25 +1908-05-05 12:34:56.1283,1908-12-25 +1908-05-05 12:34:56.1284,1908-12-25 +1908-05-05 12:34:56.1285,1908-12-25 +1908-05-05 12:34:56.1286,1908-12-25 +1908-05-05 12:34:56.1287,1908-12-25 +1908-05-05 12:34:56.1288,1908-12-25 +1908-05-05 12:34:56.1289,1908-12-25 +1908-05-05 12:34:56.129,1908-12-25 +1908-05-05 12:34:56.1291,1908-12-25 +1908-05-05 12:34:56.1292,1908-12-25 +1908-05-05 12:34:56.1293,1908-12-25 +1908-05-05 12:34:56.1294,1908-12-25 +1908-05-05 12:34:56.1295,1908-12-25 +1908-05-05 12:34:56.1296,1908-12-25 +1908-05-05 12:34:56.1297,1908-12-25 +1908-05-05 12:34:56.1298,1908-12-25 +1908-05-05 12:34:56.1299,1908-12-25 +1908-05-05 12:34:56.13,1908-12-25 +1908-05-05 12:34:56.1301,1908-12-25 +1908-05-05 12:34:56.1302,1908-12-25 +1908-05-05 12:34:56.1303,1908-12-25 +1908-05-05 12:34:56.1304,1908-12-25 +1908-05-05 12:34:56.1305,1908-12-25 +1908-05-05 12:34:56.1306,1908-12-25 +1908-05-05 12:34:56.1307,1908-12-25 +1908-05-05 12:34:56.1308,1908-12-25 +1908-05-05 12:34:56.1309,1908-12-25 +1908-05-05 12:34:56.131,1908-12-25 +1908-05-05 12:34:56.1311,1908-12-25 +1908-05-05 12:34:56.1312,1908-12-25 +1908-05-05 12:34:56.1313,1908-12-25 +1908-05-05 12:34:56.1314,1908-12-25 +1908-05-05 12:34:56.1315,1908-12-25 +1908-05-05 12:34:56.1316,1908-12-25 +1908-05-05 12:34:56.1317,1908-12-25 +1908-05-05 12:34:56.1318,1908-12-25 +1908-05-05 12:34:56.1319,1908-12-25 +1908-05-05 12:34:56.132,1908-12-25 +1908-05-05 12:34:56.1321,1908-12-25 +1908-05-05 12:34:56.1322,1908-12-25 +1908-05-05 12:34:56.1323,1908-12-25 +1908-05-05 12:34:56.1324,1908-12-25 +1908-05-05 12:34:56.1325,1908-12-25 +1908-05-05 12:34:56.1326,1908-12-25 +1908-05-05 12:34:56.1327,1908-12-25 +1908-05-05 12:34:56.1328,1908-12-25 +1908-05-05 12:34:56.1329,1908-12-25 +1908-05-05 12:34:56.133,1908-12-25 +1908-05-05 12:34:56.1331,1908-12-25 +1908-05-05 12:34:56.1332,1908-12-25 +1908-05-05 12:34:56.1333,1908-12-25 +1908-05-05 12:34:56.1334,1908-12-25 +1908-05-05 12:34:56.1335,1908-12-25 +1908-05-05 12:34:56.1336,1908-12-25 +1908-05-05 12:34:56.1337,1908-12-25 +1908-05-05 12:34:56.1338,1908-12-25 +1908-05-05 12:34:56.1339,1908-12-25 +1908-05-05 12:34:56.134,1908-12-25 +1908-05-05 12:34:56.1341,1908-12-25 +1908-05-05 12:34:56.1342,1908-12-25 +1908-05-05 12:34:56.1343,1908-12-25 +1908-05-05 12:34:56.1344,1908-12-25 +1908-05-05 12:34:56.1345,1908-12-25 +1908-05-05 12:34:56.1346,1908-12-25 +1908-05-05 12:34:56.1347,1908-12-25 +1908-05-05 12:34:56.1348,1908-12-25 +1908-05-05 12:34:56.1349,1908-12-25 +1908-05-05 12:34:56.135,1908-12-25 +1908-05-05 12:34:56.1351,1908-12-25 +1908-05-05 12:34:56.1352,1908-12-25 +1908-05-05 12:34:56.1353,1908-12-25 +1908-05-05 12:34:56.1354,1908-12-25 +1908-05-05 12:34:56.1355,1908-12-25 +1908-05-05 12:34:56.1356,1908-12-25 +1908-05-05 12:34:56.1357,1908-12-25 +1908-05-05 12:34:56.1358,1908-12-25 +1908-05-05 12:34:56.1359,1908-12-25 +1908-05-05 12:34:56.136,1908-12-25 +1908-05-05 12:34:56.1361,1908-12-25 +1908-05-05 12:34:56.1362,1908-12-25 +1908-05-05 12:34:56.1363,1908-12-25 +1908-05-05 12:34:56.1364,1908-12-25 +1908-05-05 12:34:56.1365,1908-12-25 +1908-05-05 12:34:56.1366,1908-12-25 +1908-05-05 12:34:56.1367,1908-12-25 +1908-05-05 12:34:56.1368,1908-12-25 +1908-05-05 12:34:56.1369,1908-12-25 +1908-05-05 12:34:56.137,1908-12-25 +1908-05-05 12:34:56.1371,1908-12-25 +1908-05-05 12:34:56.1372,1908-12-25 +1908-05-05 12:34:56.1373,1908-12-25 +1908-05-05 12:34:56.1374,1908-12-25 +1908-05-05 12:34:56.1375,1908-12-25 +1908-05-05 12:34:56.1376,1908-12-25 +1908-05-05 12:34:56.1377,1908-12-25 +1908-05-05 12:34:56.1378,1908-12-25 +1908-05-05 12:34:56.1379,1908-12-25 +1908-05-05 12:34:56.138,1908-12-25 +1908-05-05 12:34:56.1381,1908-12-25 +1908-05-05 12:34:56.1382,1908-12-25 +1908-05-05 12:34:56.1383,1908-12-25 +1908-05-05 12:34:56.1384,1908-12-25 +1908-05-05 12:34:56.1385,1908-12-25 +1908-05-05 12:34:56.1386,1908-12-25 +1908-05-05 12:34:56.1387,1908-12-25 +1908-05-05 12:34:56.1388,1908-12-25 +1908-05-05 12:34:56.1389,1908-12-25 +1908-05-05 12:34:56.139,1908-12-25 +1908-05-05 12:34:56.1391,1908-12-25 +1908-05-05 12:34:56.1392,1908-12-25 +1908-05-05 12:34:56.1393,1908-12-25 +1908-05-05 12:34:56.1394,1908-12-25 +1908-05-05 12:34:56.1395,1908-12-25 +1908-05-05 12:34:56.1396,1908-12-25 +1908-05-05 12:34:56.1397,1908-12-25 +1908-05-05 12:34:56.1398,1908-12-25 +1908-05-05 12:34:56.1399,1908-12-25 +1908-05-05 12:34:56.14,1908-12-25 +1908-05-05 12:34:56.1401,1908-12-25 +1908-05-05 12:34:56.1402,1908-12-25 +1908-05-05 12:34:56.1403,1908-12-25 +1908-05-05 12:34:56.1404,1908-12-25 +1908-05-05 12:34:56.1405,1908-12-25 +1908-05-05 12:34:56.1406,1908-12-25 +1908-05-05 12:34:56.1407,1908-12-25 +1908-05-05 12:34:56.1408,1908-12-25 +1908-05-05 12:34:56.1409,1908-12-25 +1908-05-05 12:34:56.141,1908-12-25 +1908-05-05 12:34:56.1411,1908-12-25 +1908-05-05 12:34:56.1412,1908-12-25 +1908-05-05 12:34:56.1413,1908-12-25 +1908-05-05 12:34:56.1414,1908-12-25 +1908-05-05 12:34:56.1415,1908-12-25 +1908-05-05 12:34:56.1416,1908-12-25 +1908-05-05 12:34:56.1417,1908-12-25 +1908-05-05 12:34:56.1418,1908-12-25 +1908-05-05 12:34:56.1419,1908-12-25 +1908-05-05 12:34:56.142,1908-12-25 +1908-05-05 12:34:56.1421,1908-12-25 +1908-05-05 12:34:56.1422,1908-12-25 +1908-05-05 12:34:56.1423,1908-12-25 +1908-05-05 12:34:56.1424,1908-12-25 +1908-05-05 12:34:56.1425,1908-12-25 +1908-05-05 12:34:56.1426,1908-12-25 +1908-05-05 12:34:56.1427,1908-12-25 +1908-05-05 12:34:56.1428,1908-12-25 +1908-05-05 12:34:56.1429,1908-12-25 +1908-05-05 12:34:56.143,1908-12-25 +1908-05-05 12:34:56.1431,1908-12-25 +1908-05-05 12:34:56.1432,1908-12-25 +1908-05-05 12:34:56.1433,1908-12-25 +1908-05-05 12:34:56.1434,1908-12-25 +1908-05-05 12:34:56.1435,1908-12-25 +1908-05-05 12:34:56.1436,1908-12-25 +1908-05-05 12:34:56.1437,1908-12-25 +1908-05-05 12:34:56.1438,1908-12-25 +1908-05-05 12:34:56.1439,1908-12-25 +1908-05-05 12:34:56.144,1908-12-25 +1908-05-05 12:34:56.1441,1908-12-25 +1908-05-05 12:34:56.1442,1908-12-25 +1908-05-05 12:34:56.1443,1908-12-25 +1908-05-05 12:34:56.1444,1908-12-25 +1908-05-05 12:34:56.1445,1908-12-25 +1908-05-05 12:34:56.1446,1908-12-25 +1908-05-05 12:34:56.1447,1908-12-25 +1908-05-05 12:34:56.1448,1908-12-25 +1908-05-05 12:34:56.1449,1908-12-25 +1908-05-05 12:34:56.145,1908-12-25 +1908-05-05 12:34:56.1451,1908-12-25 +1908-05-05 12:34:56.1452,1908-12-25 +1908-05-05 12:34:56.1453,1908-12-25 +1908-05-05 12:34:56.1454,1908-12-25 +1908-05-05 12:34:56.1455,1908-12-25 +1908-05-05 12:34:56.1456,1908-12-25 +1908-05-05 12:34:56.1457,1908-12-25 +1908-05-05 12:34:56.1458,1908-12-25 +1908-05-05 12:34:56.1459,1908-12-25 +1908-05-05 12:34:56.146,1908-12-25 +1908-05-05 12:34:56.1461,1908-12-25 +1908-05-05 12:34:56.1462,1908-12-25 +1908-05-05 12:34:56.1463,1908-12-25 +1908-05-05 12:34:56.1464,1908-12-25 +1908-05-05 12:34:56.1465,1908-12-25 +1908-05-05 12:34:56.1466,1908-12-25 +1908-05-05 12:34:56.1467,1908-12-25 +1908-05-05 12:34:56.1468,1908-12-25 +1908-05-05 12:34:56.1469,1908-12-25 +1908-05-05 12:34:56.147,1908-12-25 +1908-05-05 12:34:56.1471,1908-12-25 +1908-05-05 12:34:56.1472,1908-12-25 +1908-05-05 12:34:56.1473,1908-12-25 +1908-05-05 12:34:56.1474,1908-12-25 +1908-05-05 12:34:56.1475,1908-12-25 +1908-05-05 12:34:56.1476,1908-12-25 +1908-05-05 12:34:56.1477,1908-12-25 +1908-05-05 12:34:56.1478,1908-12-25 +1908-05-05 12:34:56.1479,1908-12-25 +1908-05-05 12:34:56.148,1908-12-25 +1908-05-05 12:34:56.1481,1908-12-25 +1908-05-05 12:34:56.1482,1908-12-25 +1908-05-05 12:34:56.1483,1908-12-25 +1908-05-05 12:34:56.1484,1908-12-25 +1908-05-05 12:34:56.1485,1908-12-25 +1908-05-05 12:34:56.1486,1908-12-25 +1908-05-05 12:34:56.1487,1908-12-25 +1908-05-05 12:34:56.1488,1908-12-25 +1908-05-05 12:34:56.1489,1908-12-25 +1908-05-05 12:34:56.149,1908-12-25 +1908-05-05 12:34:56.1491,1908-12-25 +1908-05-05 12:34:56.1492,1908-12-25 +1908-05-05 12:34:56.1493,1908-12-25 +1908-05-05 12:34:56.1494,1908-12-25 +1908-05-05 12:34:56.1495,1908-12-25 +1908-05-05 12:34:56.1496,1908-12-25 +1908-05-05 12:34:56.1497,1908-12-25 +1908-05-05 12:34:56.1498,1908-12-25 +1908-05-05 12:34:56.1499,1908-12-25 +1908-05-05 12:34:56.15,1908-12-25 +1908-05-05 12:34:56.1501,1908-12-25 +1908-05-05 12:34:56.1502,1908-12-25 +1908-05-05 12:34:56.1503,1908-12-25 +1908-05-05 12:34:56.1504,1908-12-25 +1908-05-05 12:34:56.1505,1908-12-25 +1908-05-05 12:34:56.1506,1908-12-25 +1908-05-05 12:34:56.1507,1908-12-25 +1908-05-05 12:34:56.1508,1908-12-25 +1908-05-05 12:34:56.1509,1908-12-25 +1908-05-05 12:34:56.151,1908-12-25 +1908-05-05 12:34:56.1511,1908-12-25 +1908-05-05 12:34:56.1512,1908-12-25 +1908-05-05 12:34:56.1513,1908-12-25 +1908-05-05 12:34:56.1514,1908-12-25 +1908-05-05 12:34:56.1515,1908-12-25 +1908-05-05 12:34:56.1516,1908-12-25 +1908-05-05 12:34:56.1517,1908-12-25 +1908-05-05 12:34:56.1518,1908-12-25 +1908-05-05 12:34:56.1519,1908-12-25 +1908-05-05 12:34:56.152,1908-12-25 +1908-05-05 12:34:56.1521,1908-12-25 +1908-05-05 12:34:56.1522,1908-12-25 +1908-05-05 12:34:56.1523,1908-12-25 +1908-05-05 12:34:56.1524,1908-12-25 +1908-05-05 12:34:56.1525,1908-12-25 +1908-05-05 12:34:56.1526,1908-12-25 +1908-05-05 12:34:56.1527,1908-12-25 +1908-05-05 12:34:56.1528,1908-12-25 +1908-05-05 12:34:56.1529,1908-12-25 +1908-05-05 12:34:56.153,1908-12-25 +1908-05-05 12:34:56.1531,1908-12-25 +1908-05-05 12:34:56.1532,1908-12-25 +1908-05-05 12:34:56.1533,1908-12-25 +1908-05-05 12:34:56.1534,1908-12-25 +1908-05-05 12:34:56.1535,1908-12-25 +1908-05-05 12:34:56.1536,1908-12-25 +1908-05-05 12:34:56.1537,1908-12-25 +1908-05-05 12:34:56.1538,1908-12-25 +1908-05-05 12:34:56.1539,1908-12-25 +1908-05-05 12:34:56.154,1908-12-25 +1908-05-05 12:34:56.1541,1908-12-25 +1908-05-05 12:34:56.1542,1908-12-25 +1908-05-05 12:34:56.1543,1908-12-25 +1908-05-05 12:34:56.1544,1908-12-25 +1908-05-05 12:34:56.1545,1908-12-25 +1908-05-05 12:34:56.1546,1908-12-25 +1908-05-05 12:34:56.1547,1908-12-25 +1908-05-05 12:34:56.1548,1908-12-25 +1908-05-05 12:34:56.1549,1908-12-25 +1908-05-05 12:34:56.155,1908-12-25 +1908-05-05 12:34:56.1551,1908-12-25 +1908-05-05 12:34:56.1552,1908-12-25 +1908-05-05 12:34:56.1553,1908-12-25 +1908-05-05 12:34:56.1554,1908-12-25 +1908-05-05 12:34:56.1555,1908-12-25 +1908-05-05 12:34:56.1556,1908-12-25 +1908-05-05 12:34:56.1557,1908-12-25 +1908-05-05 12:34:56.1558,1908-12-25 +1908-05-05 12:34:56.1559,1908-12-25 +1908-05-05 12:34:56.156,1908-12-25 +1908-05-05 12:34:56.1561,1908-12-25 +1908-05-05 12:34:56.1562,1908-12-25 +1908-05-05 12:34:56.1563,1908-12-25 +1908-05-05 12:34:56.1564,1908-12-25 +1908-05-05 12:34:56.1565,1908-12-25 +1908-05-05 12:34:56.1566,1908-12-25 +1908-05-05 12:34:56.1567,1908-12-25 +1908-05-05 12:34:56.1568,1908-12-25 +1908-05-05 12:34:56.1569,1908-12-25 +1908-05-05 12:34:56.157,1908-12-25 +1908-05-05 12:34:56.1571,1908-12-25 +1908-05-05 12:34:56.1572,1908-12-25 +1908-05-05 12:34:56.1573,1908-12-25 +1908-05-05 12:34:56.1574,1908-12-25 +1908-05-05 12:34:56.1575,1908-12-25 +1908-05-05 12:34:56.1576,1908-12-25 +1908-05-05 12:34:56.1577,1908-12-25 +1908-05-05 12:34:56.1578,1908-12-25 +1908-05-05 12:34:56.1579,1908-12-25 +1908-05-05 12:34:56.158,1908-12-25 +1908-05-05 12:34:56.1581,1908-12-25 +1908-05-05 12:34:56.1582,1908-12-25 +1908-05-05 12:34:56.1583,1908-12-25 +1908-05-05 12:34:56.1584,1908-12-25 +1908-05-05 12:34:56.1585,1908-12-25 +1908-05-05 12:34:56.1586,1908-12-25 +1908-05-05 12:34:56.1587,1908-12-25 +1908-05-05 12:34:56.1588,1908-12-25 +1908-05-05 12:34:56.1589,1908-12-25 +1908-05-05 12:34:56.159,1908-12-25 +1908-05-05 12:34:56.1591,1908-12-25 +1908-05-05 12:34:56.1592,1908-12-25 +1908-05-05 12:34:56.1593,1908-12-25 +1908-05-05 12:34:56.1594,1908-12-25 +1908-05-05 12:34:56.1595,1908-12-25 +1908-05-05 12:34:56.1596,1908-12-25 +1908-05-05 12:34:56.1597,1908-12-25 +1908-05-05 12:34:56.1598,1908-12-25 +1908-05-05 12:34:56.1599,1908-12-25 +1908-05-05 12:34:56.16,1908-12-25 +1908-05-05 12:34:56.1601,1908-12-25 +1908-05-05 12:34:56.1602,1908-12-25 +1908-05-05 12:34:56.1603,1908-12-25 +1908-05-05 12:34:56.1604,1908-12-25 +1908-05-05 12:34:56.1605,1908-12-25 +1908-05-05 12:34:56.1606,1908-12-25 +1908-05-05 12:34:56.1607,1908-12-25 +1908-05-05 12:34:56.1608,1908-12-25 +1908-05-05 12:34:56.1609,1908-12-25 +1908-05-05 12:34:56.161,1908-12-25 +1908-05-05 12:34:56.1611,1908-12-25 +1908-05-05 12:34:56.1612,1908-12-25 +1908-05-05 12:34:56.1613,1908-12-25 +1908-05-05 12:34:56.1614,1908-12-25 +1908-05-05 12:34:56.1615,1908-12-25 +1908-05-05 12:34:56.1616,1908-12-25 +1908-05-05 12:34:56.1617,1908-12-25 +1908-05-05 12:34:56.1618,1908-12-25 +1908-05-05 12:34:56.1619,1908-12-25 +1908-05-05 12:34:56.162,1908-12-25 +1908-05-05 12:34:56.1621,1908-12-25 +1908-05-05 12:34:56.1622,1908-12-25 +1908-05-05 12:34:56.1623,1908-12-25 +1908-05-05 12:34:56.1624,1908-12-25 +1908-05-05 12:34:56.1625,1908-12-25 +1908-05-05 12:34:56.1626,1908-12-25 +1908-05-05 12:34:56.1627,1908-12-25 +1908-05-05 12:34:56.1628,1908-12-25 +1908-05-05 12:34:56.1629,1908-12-25 +1908-05-05 12:34:56.163,1908-12-25 +1908-05-05 12:34:56.1631,1908-12-25 +1908-05-05 12:34:56.1632,1908-12-25 +1908-05-05 12:34:56.1633,1908-12-25 +1908-05-05 12:34:56.1634,1908-12-25 +1908-05-05 12:34:56.1635,1908-12-25 +1908-05-05 12:34:56.1636,1908-12-25 +1908-05-05 12:34:56.1637,1908-12-25 +1908-05-05 12:34:56.1638,1908-12-25 +1908-05-05 12:34:56.1639,1908-12-25 +1908-05-05 12:34:56.164,1908-12-25 +1908-05-05 12:34:56.1641,1908-12-25 +1908-05-05 12:34:56.1642,1908-12-25 +1908-05-05 12:34:56.1643,1908-12-25 +1908-05-05 12:34:56.1644,1908-12-25 +1908-05-05 12:34:56.1645,1908-12-25 +1908-05-05 12:34:56.1646,1908-12-25 +1908-05-05 12:34:56.1647,1908-12-25 +1908-05-05 12:34:56.1648,1908-12-25 +1908-05-05 12:34:56.1649,1908-12-25 +1908-05-05 12:34:56.165,1908-12-25 +1908-05-05 12:34:56.1651,1908-12-25 +1908-05-05 12:34:56.1652,1908-12-25 +1908-05-05 12:34:56.1653,1908-12-25 +1908-05-05 12:34:56.1654,1908-12-25 +1908-05-05 12:34:56.1655,1908-12-25 +1908-05-05 12:34:56.1656,1908-12-25 +1908-05-05 12:34:56.1657,1908-12-25 +1908-05-05 12:34:56.1658,1908-12-25 +1908-05-05 12:34:56.1659,1908-12-25 +1908-05-05 12:34:56.166,1908-12-25 +1908-05-05 12:34:56.1661,1908-12-25 +1908-05-05 12:34:56.1662,1908-12-25 +1908-05-05 12:34:56.1663,1908-12-25 +1908-05-05 12:34:56.1664,1908-12-25 +1908-05-05 12:34:56.1665,1908-12-25 +1908-05-05 12:34:56.1666,1908-12-25 +1908-05-05 12:34:56.1667,1908-12-25 +1908-05-05 12:34:56.1668,1908-12-25 +1908-05-05 12:34:56.1669,1908-12-25 +1908-05-05 12:34:56.167,1908-12-25 +1908-05-05 12:34:56.1671,1908-12-25 +1908-05-05 12:34:56.1672,1908-12-25 +1908-05-05 12:34:56.1673,1908-12-25 +1908-05-05 12:34:56.1674,1908-12-25 +1908-05-05 12:34:56.1675,1908-12-25 +1908-05-05 12:34:56.1676,1908-12-25 +1908-05-05 12:34:56.1677,1908-12-25 +1908-05-05 12:34:56.1678,1908-12-25 +1908-05-05 12:34:56.1679,1908-12-25 +1908-05-05 12:34:56.168,1908-12-25 +1908-05-05 12:34:56.1681,1908-12-25 +1908-05-05 12:34:56.1682,1908-12-25 +1908-05-05 12:34:56.1683,1908-12-25 +1908-05-05 12:34:56.1684,1908-12-25 +1908-05-05 12:34:56.1685,1908-12-25 +1908-05-05 12:34:56.1686,1908-12-25 +1908-05-05 12:34:56.1687,1908-12-25 +1908-05-05 12:34:56.1688,1908-12-25 +1908-05-05 12:34:56.1689,1908-12-25 +1908-05-05 12:34:56.169,1908-12-25 +1908-05-05 12:34:56.1691,1908-12-25 +1908-05-05 12:34:56.1692,1908-12-25 +1908-05-05 12:34:56.1693,1908-12-25 +1908-05-05 12:34:56.1694,1908-12-25 +1908-05-05 12:34:56.1695,1908-12-25 +1908-05-05 12:34:56.1696,1908-12-25 +1908-05-05 12:34:56.1697,1908-12-25 +1908-05-05 12:34:56.1698,1908-12-25 +1908-05-05 12:34:56.1699,1908-12-25 +1908-05-05 12:34:56.17,1908-12-25 +1908-05-05 12:34:56.1701,1908-12-25 +1908-05-05 12:34:56.1702,1908-12-25 +1908-05-05 12:34:56.1703,1908-12-25 +1908-05-05 12:34:56.1704,1908-12-25 +1908-05-05 12:34:56.1705,1908-12-25 +1908-05-05 12:34:56.1706,1908-12-25 +1908-05-05 12:34:56.1707,1908-12-25 +1908-05-05 12:34:56.1708,1908-12-25 +1908-05-05 12:34:56.1709,1908-12-25 +1908-05-05 12:34:56.171,1908-12-25 +1908-05-05 12:34:56.1711,1908-12-25 +1908-05-05 12:34:56.1712,1908-12-25 +1908-05-05 12:34:56.1713,1908-12-25 +1908-05-05 12:34:56.1714,1908-12-25 +1908-05-05 12:34:56.1715,1908-12-25 +1908-05-05 12:34:56.1716,1908-12-25 +1908-05-05 12:34:56.1717,1908-12-25 +1908-05-05 12:34:56.1718,1908-12-25 +1908-05-05 12:34:56.1719,1908-12-25 +1908-05-05 12:34:56.172,1908-12-25 +1908-05-05 12:34:56.1721,1908-12-25 +1908-05-05 12:34:56.1722,1908-12-25 +1908-05-05 12:34:56.1723,1908-12-25 +1908-05-05 12:34:56.1724,1908-12-25 +1908-05-05 12:34:56.1725,1908-12-25 +1908-05-05 12:34:56.1726,1908-12-25 +1908-05-05 12:34:56.1727,1908-12-25 +1908-05-05 12:34:56.1728,1908-12-25 +1908-05-05 12:34:56.1729,1908-12-25 +1908-05-05 12:34:56.173,1908-12-25 +1908-05-05 12:34:56.1731,1908-12-25 +1908-05-05 12:34:56.1732,1908-12-25 +1908-05-05 12:34:56.1733,1908-12-25 +1908-05-05 12:34:56.1734,1908-12-25 +1908-05-05 12:34:56.1735,1908-12-25 +1908-05-05 12:34:56.1736,1908-12-25 +1908-05-05 12:34:56.1737,1908-12-25 +1908-05-05 12:34:56.1738,1908-12-25 +1908-05-05 12:34:56.1739,1908-12-25 +1908-05-05 12:34:56.174,1908-12-25 +1908-05-05 12:34:56.1741,1908-12-25 +1908-05-05 12:34:56.1742,1908-12-25 +1908-05-05 12:34:56.1743,1908-12-25 +1908-05-05 12:34:56.1744,1908-12-25 +1908-05-05 12:34:56.1745,1908-12-25 +1908-05-05 12:34:56.1746,1908-12-25 +1908-05-05 12:34:56.1747,1908-12-25 +1908-05-05 12:34:56.1748,1908-12-25 +1908-05-05 12:34:56.1749,1908-12-25 +1908-05-05 12:34:56.175,1908-12-25 +1908-05-05 12:34:56.1751,1908-12-25 +1908-05-05 12:34:56.1752,1908-12-25 +1908-05-05 12:34:56.1753,1908-12-25 +1908-05-05 12:34:56.1754,1908-12-25 +1908-05-05 12:34:56.1755,1908-12-25 +1908-05-05 12:34:56.1756,1908-12-25 +1908-05-05 12:34:56.1757,1908-12-25 +1908-05-05 12:34:56.1758,1908-12-25 +1908-05-05 12:34:56.1759,1908-12-25 +1908-05-05 12:34:56.176,1908-12-25 +1908-05-05 12:34:56.1761,1908-12-25 +1908-05-05 12:34:56.1762,1908-12-25 +1908-05-05 12:34:56.1763,1908-12-25 +1908-05-05 12:34:56.1764,1908-12-25 +1908-05-05 12:34:56.1765,1908-12-25 +1908-05-05 12:34:56.1766,1908-12-25 +1908-05-05 12:34:56.1767,1908-12-25 +1908-05-05 12:34:56.1768,1908-12-25 +1908-05-05 12:34:56.1769,1908-12-25 +1908-05-05 12:34:56.177,1908-12-25 +1908-05-05 12:34:56.1771,1908-12-25 +1908-05-05 12:34:56.1772,1908-12-25 +1908-05-05 12:34:56.1773,1908-12-25 +1908-05-05 12:34:56.1774,1908-12-25 +1908-05-05 12:34:56.1775,1908-12-25 +1908-05-05 12:34:56.1776,1908-12-25 +1908-05-05 12:34:56.1777,1908-12-25 +1908-05-05 12:34:56.1778,1908-12-25 +1908-05-05 12:34:56.1779,1908-12-25 +1908-05-05 12:34:56.178,1908-12-25 +1908-05-05 12:34:56.1781,1908-12-25 +1908-05-05 12:34:56.1782,1908-12-25 +1908-05-05 12:34:56.1783,1908-12-25 +1908-05-05 12:34:56.1784,1908-12-25 +1908-05-05 12:34:56.1785,1908-12-25 +1908-05-05 12:34:56.1786,1908-12-25 +1908-05-05 12:34:56.1787,1908-12-25 +1908-05-05 12:34:56.1788,1908-12-25 +1908-05-05 12:34:56.1789,1908-12-25 +1908-05-05 12:34:56.179,1908-12-25 +1908-05-05 12:34:56.1791,1908-12-25 +1908-05-05 12:34:56.1792,1908-12-25 +1908-05-05 12:34:56.1793,1908-12-25 +1908-05-05 12:34:56.1794,1908-12-25 +1908-05-05 12:34:56.1795,1908-12-25 +1908-05-05 12:34:56.1796,1908-12-25 +1908-05-05 12:34:56.1797,1908-12-25 +1908-05-05 12:34:56.1798,1908-12-25 +1908-05-05 12:34:56.1799,1908-12-25 +1908-05-05 12:34:56.18,1908-12-25 +1908-05-05 12:34:56.1801,1908-12-25 +1908-05-05 12:34:56.1802,1908-12-25 +1908-05-05 12:34:56.1803,1908-12-25 +1908-05-05 12:34:56.1804,1908-12-25 +1908-05-05 12:34:56.1805,1908-12-25 +1908-05-05 12:34:56.1806,1908-12-25 +1908-05-05 12:34:56.1807,1908-12-25 +1908-05-05 12:34:56.1808,1908-12-25 +1908-05-05 12:34:56.1809,1908-12-25 +1908-05-05 12:34:56.181,1908-12-25 +1908-05-05 12:34:56.1811,1908-12-25 +1908-05-05 12:34:56.1812,1908-12-25 +1908-05-05 12:34:56.1813,1908-12-25 +1908-05-05 12:34:56.1814,1908-12-25 +1908-05-05 12:34:56.1815,1908-12-25 +1908-05-05 12:34:56.1816,1908-12-25 +1908-05-05 12:34:56.1817,1908-12-25 +1908-05-05 12:34:56.1818,1908-12-25 +1908-05-05 12:34:56.1819,1908-12-25 +1908-05-05 12:34:56.182,1908-12-25 +1908-05-05 12:34:56.1821,1908-12-25 +1908-05-05 12:34:56.1822,1908-12-25 +1908-05-05 12:34:56.1823,1908-12-25 +1908-05-05 12:34:56.1824,1908-12-25 +1908-05-05 12:34:56.1825,1908-12-25 +1908-05-05 12:34:56.1826,1908-12-25 +1908-05-05 12:34:56.1827,1908-12-25 +1908-05-05 12:34:56.1828,1908-12-25 +1908-05-05 12:34:56.1829,1908-12-25 +1908-05-05 12:34:56.183,1908-12-25 +1908-05-05 12:34:56.1831,1908-12-25 +1908-05-05 12:34:56.1832,1908-12-25 +1908-05-05 12:34:56.1833,1908-12-25 +1908-05-05 12:34:56.1834,1908-12-25 +1908-05-05 12:34:56.1835,1908-12-25 +1908-05-05 12:34:56.1836,1908-12-25 +1908-05-05 12:34:56.1837,1908-12-25 +1908-05-05 12:34:56.1838,1908-12-25 +1908-05-05 12:34:56.1839,1908-12-25 +1908-05-05 12:34:56.184,1908-12-25 +1908-05-05 12:34:56.1841,1908-12-25 +1908-05-05 12:34:56.1842,1908-12-25 +1908-05-05 12:34:56.1843,1908-12-25 +1908-05-05 12:34:56.1844,1908-12-25 +1908-05-05 12:34:56.1845,1908-12-25 +1908-05-05 12:34:56.1846,1908-12-25 +1908-05-05 12:34:56.1847,1908-12-25 +1908-05-05 12:34:56.1848,1908-12-25 +1908-05-05 12:34:56.1849,1908-12-25 +1908-05-05 12:34:56.185,1908-12-25 +1908-05-05 12:34:56.1851,1908-12-25 +1908-05-05 12:34:56.1852,1908-12-25 +1908-05-05 12:34:56.1853,1908-12-25 +1908-05-05 12:34:56.1854,1908-12-25 +1908-05-05 12:34:56.1855,1908-12-25 +1908-05-05 12:34:56.1856,1908-12-25 +1908-05-05 12:34:56.1857,1908-12-25 +1908-05-05 12:34:56.1858,1908-12-25 +1908-05-05 12:34:56.1859,1908-12-25 +1908-05-05 12:34:56.186,1908-12-25 +1908-05-05 12:34:56.1861,1908-12-25 +1908-05-05 12:34:56.1862,1908-12-25 +1908-05-05 12:34:56.1863,1908-12-25 +1908-05-05 12:34:56.1864,1908-12-25 +1908-05-05 12:34:56.1865,1908-12-25 +1908-05-05 12:34:56.1866,1908-12-25 +1908-05-05 12:34:56.1867,1908-12-25 +1908-05-05 12:34:56.1868,1908-12-25 +1908-05-05 12:34:56.1869,1908-12-25 +1908-05-05 12:34:56.187,1908-12-25 +1908-05-05 12:34:56.1871,1908-12-25 +1908-05-05 12:34:56.1872,1908-12-25 +1908-05-05 12:34:56.1873,1908-12-25 +1908-05-05 12:34:56.1874,1908-12-25 +1908-05-05 12:34:56.1875,1908-12-25 +1908-05-05 12:34:56.1876,1908-12-25 +1908-05-05 12:34:56.1877,1908-12-25 +1908-05-05 12:34:56.1878,1908-12-25 +1908-05-05 12:34:56.1879,1908-12-25 +1908-05-05 12:34:56.188,1908-12-25 +1908-05-05 12:34:56.1881,1908-12-25 +1908-05-05 12:34:56.1882,1908-12-25 +1908-05-05 12:34:56.1883,1908-12-25 +1908-05-05 12:34:56.1884,1908-12-25 +1908-05-05 12:34:56.1885,1908-12-25 +1908-05-05 12:34:56.1886,1908-12-25 +1908-05-05 12:34:56.1887,1908-12-25 +1908-05-05 12:34:56.1888,1908-12-25 +1908-05-05 12:34:56.1889,1908-12-25 +1908-05-05 12:34:56.189,1908-12-25 +1908-05-05 12:34:56.1891,1908-12-25 +1908-05-05 12:34:56.1892,1908-12-25 +1908-05-05 12:34:56.1893,1908-12-25 +1908-05-05 12:34:56.1894,1908-12-25 +1908-05-05 12:34:56.1895,1908-12-25 +1908-05-05 12:34:56.1896,1908-12-25 +1908-05-05 12:34:56.1897,1908-12-25 +1908-05-05 12:34:56.1898,1908-12-25 +1908-05-05 12:34:56.1899,1908-12-25 +1908-05-05 12:34:56.19,1908-12-25 +1908-05-05 12:34:56.1901,1908-12-25 +1908-05-05 12:34:56.1902,1908-12-25 +1908-05-05 12:34:56.1903,1908-12-25 +1908-05-05 12:34:56.1904,1908-12-25 +1908-05-05 12:34:56.1905,1908-12-25 +1908-05-05 12:34:56.1906,1908-12-25 +1908-05-05 12:34:56.1907,1908-12-25 +1908-05-05 12:34:56.1908,1908-12-25 +1908-05-05 12:34:56.1909,1908-12-25 +1908-05-05 12:34:56.191,1908-12-25 +1908-05-05 12:34:56.1911,1908-12-25 +1908-05-05 12:34:56.1912,1908-12-25 +1908-05-05 12:34:56.1913,1908-12-25 +1908-05-05 12:34:56.1914,1908-12-25 +1908-05-05 12:34:56.1915,1908-12-25 +1908-05-05 12:34:56.1916,1908-12-25 +1908-05-05 12:34:56.1917,1908-12-25 +1908-05-05 12:34:56.1918,1908-12-25 +1908-05-05 12:34:56.1919,1908-12-25 +1908-05-05 12:34:56.192,1908-12-25 +1908-05-05 12:34:56.1921,1908-12-25 +1908-05-05 12:34:56.1922,1908-12-25 +1908-05-05 12:34:56.1923,1908-12-25 +1908-05-05 12:34:56.1924,1908-12-25 +1908-05-05 12:34:56.1925,1908-12-25 +1908-05-05 12:34:56.1926,1908-12-25 +1908-05-05 12:34:56.1927,1908-12-25 +1908-05-05 12:34:56.1928,1908-12-25 +1908-05-05 12:34:56.1929,1908-12-25 +1908-05-05 12:34:56.193,1908-12-25 +1908-05-05 12:34:56.1931,1908-12-25 +1908-05-05 12:34:56.1932,1908-12-25 +1908-05-05 12:34:56.1933,1908-12-25 +1908-05-05 12:34:56.1934,1908-12-25 +1908-05-05 12:34:56.1935,1908-12-25 +1908-05-05 12:34:56.1936,1908-12-25 +1908-05-05 12:34:56.1937,1908-12-25 +1908-05-05 12:34:56.1938,1908-12-25 +1908-05-05 12:34:56.1939,1908-12-25 +1908-05-05 12:34:56.194,1908-12-25 +1908-05-05 12:34:56.1941,1908-12-25 +1908-05-05 12:34:56.1942,1908-12-25 +1908-05-05 12:34:56.1943,1908-12-25 +1908-05-05 12:34:56.1944,1908-12-25 +1908-05-05 12:34:56.1945,1908-12-25 +1908-05-05 12:34:56.1946,1908-12-25 +1908-05-05 12:34:56.1947,1908-12-25 +1908-05-05 12:34:56.1948,1908-12-25 +1908-05-05 12:34:56.1949,1908-12-25 +1908-05-05 12:34:56.195,1908-12-25 +1908-05-05 12:34:56.1951,1908-12-25 +1908-05-05 12:34:56.1952,1908-12-25 +1908-05-05 12:34:56.1953,1908-12-25 +1908-05-05 12:34:56.1954,1908-12-25 +1908-05-05 12:34:56.1955,1908-12-25 +1908-05-05 12:34:56.1956,1908-12-25 +1908-05-05 12:34:56.1957,1908-12-25 +1908-05-05 12:34:56.1958,1908-12-25 +1908-05-05 12:34:56.1959,1908-12-25 +1908-05-05 12:34:56.196,1908-12-25 +1908-05-05 12:34:56.1961,1908-12-25 +1908-05-05 12:34:56.1962,1908-12-25 +1908-05-05 12:34:56.1963,1908-12-25 +1908-05-05 12:34:56.1964,1908-12-25 +1908-05-05 12:34:56.1965,1908-12-25 +1908-05-05 12:34:56.1966,1908-12-25 +1908-05-05 12:34:56.1967,1908-12-25 +1908-05-05 12:34:56.1968,1908-12-25 +1908-05-05 12:34:56.1969,1908-12-25 +1908-05-05 12:34:56.197,1908-12-25 +1908-05-05 12:34:56.1971,1908-12-25 +1908-05-05 12:34:56.1972,1908-12-25 +1908-05-05 12:34:56.1973,1908-12-25 +1908-05-05 12:34:56.1974,1908-12-25 +1908-05-05 12:34:56.1975,1908-12-25 +1908-05-05 12:34:56.1976,1908-12-25 +1908-05-05 12:34:56.1977,1908-12-25 +1908-05-05 12:34:56.1978,1908-12-25 +1908-05-05 12:34:56.1979,1908-12-25 +1908-05-05 12:34:56.198,1908-12-25 +1908-05-05 12:34:56.1981,1908-12-25 +1908-05-05 12:34:56.1982,1908-12-25 +1908-05-05 12:34:56.1983,1908-12-25 +1908-05-05 12:34:56.1984,1908-12-25 +1908-05-05 12:34:56.1985,1908-12-25 +1908-05-05 12:34:56.1986,1908-12-25 +1908-05-05 12:34:56.1987,1908-12-25 +1908-05-05 12:34:56.1988,1908-12-25 +1908-05-05 12:34:56.1989,1908-12-25 +1908-05-05 12:34:56.199,1908-12-25 +1908-05-05 12:34:56.1991,1908-12-25 +1908-05-05 12:34:56.1992,1908-12-25 +1908-05-05 12:34:56.1993,1908-12-25 +1908-05-05 12:34:56.1994,1908-12-25 +1908-05-05 12:34:56.1995,1908-12-25 +1908-05-05 12:34:56.1996,1908-12-25 +1908-05-05 12:34:56.1997,1908-12-25 +1908-05-05 12:34:56.1998,1908-12-25 +1908-05-05 12:34:56.1999,1908-12-25 +1909-05-05 12:34:56.1,1909-12-25 +1909-05-05 12:34:56.1001,1909-12-25 +1909-05-05 12:34:56.1002,1909-12-25 +1909-05-05 12:34:56.1003,1909-12-25 +1909-05-05 12:34:56.1004,1909-12-25 +1909-05-05 12:34:56.1005,1909-12-25 +1909-05-05 12:34:56.1006,1909-12-25 +1909-05-05 12:34:56.1007,1909-12-25 +1909-05-05 12:34:56.1008,1909-12-25 +1909-05-05 12:34:56.1009,1909-12-25 +1909-05-05 12:34:56.101,1909-12-25 +1909-05-05 12:34:56.1011,1909-12-25 +1909-05-05 12:34:56.1012,1909-12-25 +1909-05-05 12:34:56.1013,1909-12-25 +1909-05-05 12:34:56.1014,1909-12-25 +1909-05-05 12:34:56.1015,1909-12-25 +1909-05-05 12:34:56.1016,1909-12-25 +1909-05-05 12:34:56.1017,1909-12-25 +1909-05-05 12:34:56.1018,1909-12-25 +1909-05-05 12:34:56.1019,1909-12-25 +1909-05-05 12:34:56.102,1909-12-25 +1909-05-05 12:34:56.1021,1909-12-25 +1909-05-05 12:34:56.1022,1909-12-25 +1909-05-05 12:34:56.1023,1909-12-25 +1909-05-05 12:34:56.1024,1909-12-25 +1909-05-05 12:34:56.1025,1909-12-25 +1909-05-05 12:34:56.1026,1909-12-25 +1909-05-05 12:34:56.1027,1909-12-25 +1909-05-05 12:34:56.1028,1909-12-25 +1909-05-05 12:34:56.1029,1909-12-25 +1909-05-05 12:34:56.103,1909-12-25 +1909-05-05 12:34:56.1031,1909-12-25 +1909-05-05 12:34:56.1032,1909-12-25 +1909-05-05 12:34:56.1033,1909-12-25 +1909-05-05 12:34:56.1034,1909-12-25 +1909-05-05 12:34:56.1035,1909-12-25 +1909-05-05 12:34:56.1036,1909-12-25 +1909-05-05 12:34:56.1037,1909-12-25 +1909-05-05 12:34:56.1038,1909-12-25 +1909-05-05 12:34:56.1039,1909-12-25 +1909-05-05 12:34:56.104,1909-12-25 +1909-05-05 12:34:56.1041,1909-12-25 +1909-05-05 12:34:56.1042,1909-12-25 +1909-05-05 12:34:56.1043,1909-12-25 +1909-05-05 12:34:56.1044,1909-12-25 +1909-05-05 12:34:56.1045,1909-12-25 +1909-05-05 12:34:56.1046,1909-12-25 +1909-05-05 12:34:56.1047,1909-12-25 +1909-05-05 12:34:56.1048,1909-12-25 +1909-05-05 12:34:56.1049,1909-12-25 +1909-05-05 12:34:56.105,1909-12-25 +1909-05-05 12:34:56.1051,1909-12-25 +1909-05-05 12:34:56.1052,1909-12-25 +1909-05-05 12:34:56.1053,1909-12-25 +1909-05-05 12:34:56.1054,1909-12-25 +1909-05-05 12:34:56.1055,1909-12-25 +1909-05-05 12:34:56.1056,1909-12-25 +1909-05-05 12:34:56.1057,1909-12-25 +1909-05-05 12:34:56.1058,1909-12-25 +1909-05-05 12:34:56.1059,1909-12-25 +1909-05-05 12:34:56.106,1909-12-25 +1909-05-05 12:34:56.1061,1909-12-25 +1909-05-05 12:34:56.1062,1909-12-25 +1909-05-05 12:34:56.1063,1909-12-25 +1909-05-05 12:34:56.1064,1909-12-25 +1909-05-05 12:34:56.1065,1909-12-25 +1909-05-05 12:34:56.1066,1909-12-25 +1909-05-05 12:34:56.1067,1909-12-25 +1909-05-05 12:34:56.1068,1909-12-25 +1909-05-05 12:34:56.1069,1909-12-25 +1909-05-05 12:34:56.107,1909-12-25 +1909-05-05 12:34:56.1071,1909-12-25 +1909-05-05 12:34:56.1072,1909-12-25 +1909-05-05 12:34:56.1073,1909-12-25 +1909-05-05 12:34:56.1074,1909-12-25 +1909-05-05 12:34:56.1075,1909-12-25 +1909-05-05 12:34:56.1076,1909-12-25 +1909-05-05 12:34:56.1077,1909-12-25 +1909-05-05 12:34:56.1078,1909-12-25 +1909-05-05 12:34:56.1079,1909-12-25 +1909-05-05 12:34:56.108,1909-12-25 +1909-05-05 12:34:56.1081,1909-12-25 +1909-05-05 12:34:56.1082,1909-12-25 +1909-05-05 12:34:56.1083,1909-12-25 +1909-05-05 12:34:56.1084,1909-12-25 +1909-05-05 12:34:56.1085,1909-12-25 +1909-05-05 12:34:56.1086,1909-12-25 +1909-05-05 12:34:56.1087,1909-12-25 +1909-05-05 12:34:56.1088,1909-12-25 +1909-05-05 12:34:56.1089,1909-12-25 +1909-05-05 12:34:56.109,1909-12-25 +1909-05-05 12:34:56.1091,1909-12-25 +1909-05-05 12:34:56.1092,1909-12-25 +1909-05-05 12:34:56.1093,1909-12-25 +1909-05-05 12:34:56.1094,1909-12-25 +1909-05-05 12:34:56.1095,1909-12-25 +1909-05-05 12:34:56.1096,1909-12-25 +1909-05-05 12:34:56.1097,1909-12-25 +1909-05-05 12:34:56.1098,1909-12-25 +1909-05-05 12:34:56.1099,1909-12-25 +1909-05-05 12:34:56.11,1909-12-25 +1909-05-05 12:34:56.1101,1909-12-25 +1909-05-05 12:34:56.1102,1909-12-25 +1909-05-05 12:34:56.1103,1909-12-25 +1909-05-05 12:34:56.1104,1909-12-25 +1909-05-05 12:34:56.1105,1909-12-25 +1909-05-05 12:34:56.1106,1909-12-25 +1909-05-05 12:34:56.1107,1909-12-25 +1909-05-05 12:34:56.1108,1909-12-25 +1909-05-05 12:34:56.1109,1909-12-25 +1909-05-05 12:34:56.111,1909-12-25 +1909-05-05 12:34:56.1111,1909-12-25 +1909-05-05 12:34:56.1112,1909-12-25 +1909-05-05 12:34:56.1113,1909-12-25 +1909-05-05 12:34:56.1114,1909-12-25 +1909-05-05 12:34:56.1115,1909-12-25 +1909-05-05 12:34:56.1116,1909-12-25 +1909-05-05 12:34:56.1117,1909-12-25 +1909-05-05 12:34:56.1118,1909-12-25 +1909-05-05 12:34:56.1119,1909-12-25 +1909-05-05 12:34:56.112,1909-12-25 +1909-05-05 12:34:56.1121,1909-12-25 +1909-05-05 12:34:56.1122,1909-12-25 +1909-05-05 12:34:56.1123,1909-12-25 +1909-05-05 12:34:56.1124,1909-12-25 +1909-05-05 12:34:56.1125,1909-12-25 +1909-05-05 12:34:56.1126,1909-12-25 +1909-05-05 12:34:56.1127,1909-12-25 +1909-05-05 12:34:56.1128,1909-12-25 +1909-05-05 12:34:56.1129,1909-12-25 +1909-05-05 12:34:56.113,1909-12-25 +1909-05-05 12:34:56.1131,1909-12-25 +1909-05-05 12:34:56.1132,1909-12-25 +1909-05-05 12:34:56.1133,1909-12-25 +1909-05-05 12:34:56.1134,1909-12-25 +1909-05-05 12:34:56.1135,1909-12-25 +1909-05-05 12:34:56.1136,1909-12-25 +1909-05-05 12:34:56.1137,1909-12-25 +1909-05-05 12:34:56.1138,1909-12-25 +1909-05-05 12:34:56.1139,1909-12-25 +1909-05-05 12:34:56.114,1909-12-25 +1909-05-05 12:34:56.1141,1909-12-25 +1909-05-05 12:34:56.1142,1909-12-25 +1909-05-05 12:34:56.1143,1909-12-25 +1909-05-05 12:34:56.1144,1909-12-25 +1909-05-05 12:34:56.1145,1909-12-25 +1909-05-05 12:34:56.1146,1909-12-25 +1909-05-05 12:34:56.1147,1909-12-25 +1909-05-05 12:34:56.1148,1909-12-25 +1909-05-05 12:34:56.1149,1909-12-25 +1909-05-05 12:34:56.115,1909-12-25 +1909-05-05 12:34:56.1151,1909-12-25 +1909-05-05 12:34:56.1152,1909-12-25 +1909-05-05 12:34:56.1153,1909-12-25 +1909-05-05 12:34:56.1154,1909-12-25 +1909-05-05 12:34:56.1155,1909-12-25 +1909-05-05 12:34:56.1156,1909-12-25 +1909-05-05 12:34:56.1157,1909-12-25 +1909-05-05 12:34:56.1158,1909-12-25 +1909-05-05 12:34:56.1159,1909-12-25 +1909-05-05 12:34:56.116,1909-12-25 +1909-05-05 12:34:56.1161,1909-12-25 +1909-05-05 12:34:56.1162,1909-12-25 +1909-05-05 12:34:56.1163,1909-12-25 +1909-05-05 12:34:56.1164,1909-12-25 +1909-05-05 12:34:56.1165,1909-12-25 +1909-05-05 12:34:56.1166,1909-12-25 +1909-05-05 12:34:56.1167,1909-12-25 +1909-05-05 12:34:56.1168,1909-12-25 +1909-05-05 12:34:56.1169,1909-12-25 +1909-05-05 12:34:56.117,1909-12-25 +1909-05-05 12:34:56.1171,1909-12-25 +1909-05-05 12:34:56.1172,1909-12-25 +1909-05-05 12:34:56.1173,1909-12-25 +1909-05-05 12:34:56.1174,1909-12-25 +1909-05-05 12:34:56.1175,1909-12-25 +1909-05-05 12:34:56.1176,1909-12-25 +1909-05-05 12:34:56.1177,1909-12-25 +1909-05-05 12:34:56.1178,1909-12-25 +1909-05-05 12:34:56.1179,1909-12-25 +1909-05-05 12:34:56.118,1909-12-25 +1909-05-05 12:34:56.1181,1909-12-25 +1909-05-05 12:34:56.1182,1909-12-25 +1909-05-05 12:34:56.1183,1909-12-25 +1909-05-05 12:34:56.1184,1909-12-25 +1909-05-05 12:34:56.1185,1909-12-25 +1909-05-05 12:34:56.1186,1909-12-25 +1909-05-05 12:34:56.1187,1909-12-25 +1909-05-05 12:34:56.1188,1909-12-25 +1909-05-05 12:34:56.1189,1909-12-25 +1909-05-05 12:34:56.119,1909-12-25 +1909-05-05 12:34:56.1191,1909-12-25 +1909-05-05 12:34:56.1192,1909-12-25 +1909-05-05 12:34:56.1193,1909-12-25 +1909-05-05 12:34:56.1194,1909-12-25 +1909-05-05 12:34:56.1195,1909-12-25 +1909-05-05 12:34:56.1196,1909-12-25 +1909-05-05 12:34:56.1197,1909-12-25 +1909-05-05 12:34:56.1198,1909-12-25 +1909-05-05 12:34:56.1199,1909-12-25 +1909-05-05 12:34:56.12,1909-12-25 +1909-05-05 12:34:56.1201,1909-12-25 +1909-05-05 12:34:56.1202,1909-12-25 +1909-05-05 12:34:56.1203,1909-12-25 +1909-05-05 12:34:56.1204,1909-12-25 +1909-05-05 12:34:56.1205,1909-12-25 +1909-05-05 12:34:56.1206,1909-12-25 +1909-05-05 12:34:56.1207,1909-12-25 +1909-05-05 12:34:56.1208,1909-12-25 +1909-05-05 12:34:56.1209,1909-12-25 +1909-05-05 12:34:56.121,1909-12-25 +1909-05-05 12:34:56.1211,1909-12-25 +1909-05-05 12:34:56.1212,1909-12-25 +1909-05-05 12:34:56.1213,1909-12-25 +1909-05-05 12:34:56.1214,1909-12-25 +1909-05-05 12:34:56.1215,1909-12-25 +1909-05-05 12:34:56.1216,1909-12-25 +1909-05-05 12:34:56.1217,1909-12-25 +1909-05-05 12:34:56.1218,1909-12-25 +1909-05-05 12:34:56.1219,1909-12-25 +1909-05-05 12:34:56.122,1909-12-25 +1909-05-05 12:34:56.1221,1909-12-25 +1909-05-05 12:34:56.1222,1909-12-25 +1909-05-05 12:34:56.1223,1909-12-25 +1909-05-05 12:34:56.1224,1909-12-25 +1909-05-05 12:34:56.1225,1909-12-25 +1909-05-05 12:34:56.1226,1909-12-25 +1909-05-05 12:34:56.1227,1909-12-25 +1909-05-05 12:34:56.1228,1909-12-25 +1909-05-05 12:34:56.1229,1909-12-25 +1909-05-05 12:34:56.123,1909-12-25 +1909-05-05 12:34:56.1231,1909-12-25 +1909-05-05 12:34:56.1232,1909-12-25 +1909-05-05 12:34:56.1233,1909-12-25 +1909-05-05 12:34:56.1234,1909-12-25 +1909-05-05 12:34:56.1235,1909-12-25 +1909-05-05 12:34:56.1236,1909-12-25 +1909-05-05 12:34:56.1237,1909-12-25 +1909-05-05 12:34:56.1238,1909-12-25 +1909-05-05 12:34:56.1239,1909-12-25 +1909-05-05 12:34:56.124,1909-12-25 +1909-05-05 12:34:56.1241,1909-12-25 +1909-05-05 12:34:56.1242,1909-12-25 +1909-05-05 12:34:56.1243,1909-12-25 +1909-05-05 12:34:56.1244,1909-12-25 +1909-05-05 12:34:56.1245,1909-12-25 +1909-05-05 12:34:56.1246,1909-12-25 +1909-05-05 12:34:56.1247,1909-12-25 +1909-05-05 12:34:56.1248,1909-12-25 +1909-05-05 12:34:56.1249,1909-12-25 +1909-05-05 12:34:56.125,1909-12-25 +1909-05-05 12:34:56.1251,1909-12-25 +1909-05-05 12:34:56.1252,1909-12-25 +1909-05-05 12:34:56.1253,1909-12-25 +1909-05-05 12:34:56.1254,1909-12-25 +1909-05-05 12:34:56.1255,1909-12-25 +1909-05-05 12:34:56.1256,1909-12-25 +1909-05-05 12:34:56.1257,1909-12-25 +1909-05-05 12:34:56.1258,1909-12-25 +1909-05-05 12:34:56.1259,1909-12-25 +1909-05-05 12:34:56.126,1909-12-25 +1909-05-05 12:34:56.1261,1909-12-25 +1909-05-05 12:34:56.1262,1909-12-25 +1909-05-05 12:34:56.1263,1909-12-25 +1909-05-05 12:34:56.1264,1909-12-25 +1909-05-05 12:34:56.1265,1909-12-25 +1909-05-05 12:34:56.1266,1909-12-25 +1909-05-05 12:34:56.1267,1909-12-25 +1909-05-05 12:34:56.1268,1909-12-25 +1909-05-05 12:34:56.1269,1909-12-25 +1909-05-05 12:34:56.127,1909-12-25 +1909-05-05 12:34:56.1271,1909-12-25 +1909-05-05 12:34:56.1272,1909-12-25 +1909-05-05 12:34:56.1273,1909-12-25 +1909-05-05 12:34:56.1274,1909-12-25 +1909-05-05 12:34:56.1275,1909-12-25 +1909-05-05 12:34:56.1276,1909-12-25 +1909-05-05 12:34:56.1277,1909-12-25 +1909-05-05 12:34:56.1278,1909-12-25 +1909-05-05 12:34:56.1279,1909-12-25 +1909-05-05 12:34:56.128,1909-12-25 +1909-05-05 12:34:56.1281,1909-12-25 +1909-05-05 12:34:56.1282,1909-12-25 +1909-05-05 12:34:56.1283,1909-12-25 +1909-05-05 12:34:56.1284,1909-12-25 +1909-05-05 12:34:56.1285,1909-12-25 +1909-05-05 12:34:56.1286,1909-12-25 +1909-05-05 12:34:56.1287,1909-12-25 +1909-05-05 12:34:56.1288,1909-12-25 +1909-05-05 12:34:56.1289,1909-12-25 +1909-05-05 12:34:56.129,1909-12-25 +1909-05-05 12:34:56.1291,1909-12-25 +1909-05-05 12:34:56.1292,1909-12-25 +1909-05-05 12:34:56.1293,1909-12-25 +1909-05-05 12:34:56.1294,1909-12-25 +1909-05-05 12:34:56.1295,1909-12-25 +1909-05-05 12:34:56.1296,1909-12-25 +1909-05-05 12:34:56.1297,1909-12-25 +1909-05-05 12:34:56.1298,1909-12-25 +1909-05-05 12:34:56.1299,1909-12-25 +1909-05-05 12:34:56.13,1909-12-25 +1909-05-05 12:34:56.1301,1909-12-25 +1909-05-05 12:34:56.1302,1909-12-25 +1909-05-05 12:34:56.1303,1909-12-25 +1909-05-05 12:34:56.1304,1909-12-25 +1909-05-05 12:34:56.1305,1909-12-25 +1909-05-05 12:34:56.1306,1909-12-25 +1909-05-05 12:34:56.1307,1909-12-25 +1909-05-05 12:34:56.1308,1909-12-25 +1909-05-05 12:34:56.1309,1909-12-25 +1909-05-05 12:34:56.131,1909-12-25 +1909-05-05 12:34:56.1311,1909-12-25 +1909-05-05 12:34:56.1312,1909-12-25 +1909-05-05 12:34:56.1313,1909-12-25 +1909-05-05 12:34:56.1314,1909-12-25 +1909-05-05 12:34:56.1315,1909-12-25 +1909-05-05 12:34:56.1316,1909-12-25 +1909-05-05 12:34:56.1317,1909-12-25 +1909-05-05 12:34:56.1318,1909-12-25 +1909-05-05 12:34:56.1319,1909-12-25 +1909-05-05 12:34:56.132,1909-12-25 +1909-05-05 12:34:56.1321,1909-12-25 +1909-05-05 12:34:56.1322,1909-12-25 +1909-05-05 12:34:56.1323,1909-12-25 +1909-05-05 12:34:56.1324,1909-12-25 +1909-05-05 12:34:56.1325,1909-12-25 +1909-05-05 12:34:56.1326,1909-12-25 +1909-05-05 12:34:56.1327,1909-12-25 +1909-05-05 12:34:56.1328,1909-12-25 +1909-05-05 12:34:56.1329,1909-12-25 +1909-05-05 12:34:56.133,1909-12-25 +1909-05-05 12:34:56.1331,1909-12-25 +1909-05-05 12:34:56.1332,1909-12-25 +1909-05-05 12:34:56.1333,1909-12-25 +1909-05-05 12:34:56.1334,1909-12-25 +1909-05-05 12:34:56.1335,1909-12-25 +1909-05-05 12:34:56.1336,1909-12-25 +1909-05-05 12:34:56.1337,1909-12-25 +1909-05-05 12:34:56.1338,1909-12-25 +1909-05-05 12:34:56.1339,1909-12-25 +1909-05-05 12:34:56.134,1909-12-25 +1909-05-05 12:34:56.1341,1909-12-25 +1909-05-05 12:34:56.1342,1909-12-25 +1909-05-05 12:34:56.1343,1909-12-25 +1909-05-05 12:34:56.1344,1909-12-25 +1909-05-05 12:34:56.1345,1909-12-25 +1909-05-05 12:34:56.1346,1909-12-25 +1909-05-05 12:34:56.1347,1909-12-25 +1909-05-05 12:34:56.1348,1909-12-25 +1909-05-05 12:34:56.1349,1909-12-25 +1909-05-05 12:34:56.135,1909-12-25 +1909-05-05 12:34:56.1351,1909-12-25 +1909-05-05 12:34:56.1352,1909-12-25 +1909-05-05 12:34:56.1353,1909-12-25 +1909-05-05 12:34:56.1354,1909-12-25 +1909-05-05 12:34:56.1355,1909-12-25 +1909-05-05 12:34:56.1356,1909-12-25 +1909-05-05 12:34:56.1357,1909-12-25 +1909-05-05 12:34:56.1358,1909-12-25 +1909-05-05 12:34:56.1359,1909-12-25 +1909-05-05 12:34:56.136,1909-12-25 +1909-05-05 12:34:56.1361,1909-12-25 +1909-05-05 12:34:56.1362,1909-12-25 +1909-05-05 12:34:56.1363,1909-12-25 +1909-05-05 12:34:56.1364,1909-12-25 +1909-05-05 12:34:56.1365,1909-12-25 +1909-05-05 12:34:56.1366,1909-12-25 +1909-05-05 12:34:56.1367,1909-12-25 +1909-05-05 12:34:56.1368,1909-12-25 +1909-05-05 12:34:56.1369,1909-12-25 +1909-05-05 12:34:56.137,1909-12-25 +1909-05-05 12:34:56.1371,1909-12-25 +1909-05-05 12:34:56.1372,1909-12-25 +1909-05-05 12:34:56.1373,1909-12-25 +1909-05-05 12:34:56.1374,1909-12-25 +1909-05-05 12:34:56.1375,1909-12-25 +1909-05-05 12:34:56.1376,1909-12-25 +1909-05-05 12:34:56.1377,1909-12-25 +1909-05-05 12:34:56.1378,1909-12-25 +1909-05-05 12:34:56.1379,1909-12-25 +1909-05-05 12:34:56.138,1909-12-25 +1909-05-05 12:34:56.1381,1909-12-25 +1909-05-05 12:34:56.1382,1909-12-25 +1909-05-05 12:34:56.1383,1909-12-25 +1909-05-05 12:34:56.1384,1909-12-25 +1909-05-05 12:34:56.1385,1909-12-25 +1909-05-05 12:34:56.1386,1909-12-25 +1909-05-05 12:34:56.1387,1909-12-25 +1909-05-05 12:34:56.1388,1909-12-25 +1909-05-05 12:34:56.1389,1909-12-25 +1909-05-05 12:34:56.139,1909-12-25 +1909-05-05 12:34:56.1391,1909-12-25 +1909-05-05 12:34:56.1392,1909-12-25 +1909-05-05 12:34:56.1393,1909-12-25 +1909-05-05 12:34:56.1394,1909-12-25 +1909-05-05 12:34:56.1395,1909-12-25 +1909-05-05 12:34:56.1396,1909-12-25 +1909-05-05 12:34:56.1397,1909-12-25 +1909-05-05 12:34:56.1398,1909-12-25 +1909-05-05 12:34:56.1399,1909-12-25 +1909-05-05 12:34:56.14,1909-12-25 +1909-05-05 12:34:56.1401,1909-12-25 +1909-05-05 12:34:56.1402,1909-12-25 +1909-05-05 12:34:56.1403,1909-12-25 +1909-05-05 12:34:56.1404,1909-12-25 +1909-05-05 12:34:56.1405,1909-12-25 +1909-05-05 12:34:56.1406,1909-12-25 +1909-05-05 12:34:56.1407,1909-12-25 +1909-05-05 12:34:56.1408,1909-12-25 +1909-05-05 12:34:56.1409,1909-12-25 +1909-05-05 12:34:56.141,1909-12-25 +1909-05-05 12:34:56.1411,1909-12-25 +1909-05-05 12:34:56.1412,1909-12-25 +1909-05-05 12:34:56.1413,1909-12-25 +1909-05-05 12:34:56.1414,1909-12-25 +1909-05-05 12:34:56.1415,1909-12-25 +1909-05-05 12:34:56.1416,1909-12-25 +1909-05-05 12:34:56.1417,1909-12-25 +1909-05-05 12:34:56.1418,1909-12-25 +1909-05-05 12:34:56.1419,1909-12-25 +1909-05-05 12:34:56.142,1909-12-25 +1909-05-05 12:34:56.1421,1909-12-25 +1909-05-05 12:34:56.1422,1909-12-25 +1909-05-05 12:34:56.1423,1909-12-25 +1909-05-05 12:34:56.1424,1909-12-25 +1909-05-05 12:34:56.1425,1909-12-25 +1909-05-05 12:34:56.1426,1909-12-25 +1909-05-05 12:34:56.1427,1909-12-25 +1909-05-05 12:34:56.1428,1909-12-25 +1909-05-05 12:34:56.1429,1909-12-25 +1909-05-05 12:34:56.143,1909-12-25 +1909-05-05 12:34:56.1431,1909-12-25 +1909-05-05 12:34:56.1432,1909-12-25 +1909-05-05 12:34:56.1433,1909-12-25 +1909-05-05 12:34:56.1434,1909-12-25 +1909-05-05 12:34:56.1435,1909-12-25 +1909-05-05 12:34:56.1436,1909-12-25 +1909-05-05 12:34:56.1437,1909-12-25 +1909-05-05 12:34:56.1438,1909-12-25 +1909-05-05 12:34:56.1439,1909-12-25 +1909-05-05 12:34:56.144,1909-12-25 +1909-05-05 12:34:56.1441,1909-12-25 +1909-05-05 12:34:56.1442,1909-12-25 +1909-05-05 12:34:56.1443,1909-12-25 +1909-05-05 12:34:56.1444,1909-12-25 +1909-05-05 12:34:56.1445,1909-12-25 +1909-05-05 12:34:56.1446,1909-12-25 +1909-05-05 12:34:56.1447,1909-12-25 +1909-05-05 12:34:56.1448,1909-12-25 +1909-05-05 12:34:56.1449,1909-12-25 +1909-05-05 12:34:56.145,1909-12-25 +1909-05-05 12:34:56.1451,1909-12-25 +1909-05-05 12:34:56.1452,1909-12-25 +1909-05-05 12:34:56.1453,1909-12-25 +1909-05-05 12:34:56.1454,1909-12-25 +1909-05-05 12:34:56.1455,1909-12-25 +1909-05-05 12:34:56.1456,1909-12-25 +1909-05-05 12:34:56.1457,1909-12-25 +1909-05-05 12:34:56.1458,1909-12-25 +1909-05-05 12:34:56.1459,1909-12-25 +1909-05-05 12:34:56.146,1909-12-25 +1909-05-05 12:34:56.1461,1909-12-25 +1909-05-05 12:34:56.1462,1909-12-25 +1909-05-05 12:34:56.1463,1909-12-25 +1909-05-05 12:34:56.1464,1909-12-25 +1909-05-05 12:34:56.1465,1909-12-25 +1909-05-05 12:34:56.1466,1909-12-25 +1909-05-05 12:34:56.1467,1909-12-25 +1909-05-05 12:34:56.1468,1909-12-25 +1909-05-05 12:34:56.1469,1909-12-25 +1909-05-05 12:34:56.147,1909-12-25 +1909-05-05 12:34:56.1471,1909-12-25 +1909-05-05 12:34:56.1472,1909-12-25 +1909-05-05 12:34:56.1473,1909-12-25 +1909-05-05 12:34:56.1474,1909-12-25 +1909-05-05 12:34:56.1475,1909-12-25 +1909-05-05 12:34:56.1476,1909-12-25 +1909-05-05 12:34:56.1477,1909-12-25 +1909-05-05 12:34:56.1478,1909-12-25 +1909-05-05 12:34:56.1479,1909-12-25 +1909-05-05 12:34:56.148,1909-12-25 +1909-05-05 12:34:56.1481,1909-12-25 +1909-05-05 12:34:56.1482,1909-12-25 +1909-05-05 12:34:56.1483,1909-12-25 +1909-05-05 12:34:56.1484,1909-12-25 +1909-05-05 12:34:56.1485,1909-12-25 +1909-05-05 12:34:56.1486,1909-12-25 +1909-05-05 12:34:56.1487,1909-12-25 +1909-05-05 12:34:56.1488,1909-12-25 +1909-05-05 12:34:56.1489,1909-12-25 +1909-05-05 12:34:56.149,1909-12-25 +1909-05-05 12:34:56.1491,1909-12-25 +1909-05-05 12:34:56.1492,1909-12-25 +1909-05-05 12:34:56.1493,1909-12-25 +1909-05-05 12:34:56.1494,1909-12-25 +1909-05-05 12:34:56.1495,1909-12-25 +1909-05-05 12:34:56.1496,1909-12-25 +1909-05-05 12:34:56.1497,1909-12-25 +1909-05-05 12:34:56.1498,1909-12-25 +1909-05-05 12:34:56.1499,1909-12-25 +1909-05-05 12:34:56.15,1909-12-25 +1909-05-05 12:34:56.1501,1909-12-25 +1909-05-05 12:34:56.1502,1909-12-25 +1909-05-05 12:34:56.1503,1909-12-25 +1909-05-05 12:34:56.1504,1909-12-25 +1909-05-05 12:34:56.1505,1909-12-25 +1909-05-05 12:34:56.1506,1909-12-25 +1909-05-05 12:34:56.1507,1909-12-25 +1909-05-05 12:34:56.1508,1909-12-25 +1909-05-05 12:34:56.1509,1909-12-25 +1909-05-05 12:34:56.151,1909-12-25 +1909-05-05 12:34:56.1511,1909-12-25 +1909-05-05 12:34:56.1512,1909-12-25 +1909-05-05 12:34:56.1513,1909-12-25 +1909-05-05 12:34:56.1514,1909-12-25 +1909-05-05 12:34:56.1515,1909-12-25 +1909-05-05 12:34:56.1516,1909-12-25 +1909-05-05 12:34:56.1517,1909-12-25 +1909-05-05 12:34:56.1518,1909-12-25 +1909-05-05 12:34:56.1519,1909-12-25 +1909-05-05 12:34:56.152,1909-12-25 +1909-05-05 12:34:56.1521,1909-12-25 +1909-05-05 12:34:56.1522,1909-12-25 +1909-05-05 12:34:56.1523,1909-12-25 +1909-05-05 12:34:56.1524,1909-12-25 +1909-05-05 12:34:56.1525,1909-12-25 +1909-05-05 12:34:56.1526,1909-12-25 +1909-05-05 12:34:56.1527,1909-12-25 +1909-05-05 12:34:56.1528,1909-12-25 +1909-05-05 12:34:56.1529,1909-12-25 +1909-05-05 12:34:56.153,1909-12-25 +1909-05-05 12:34:56.1531,1909-12-25 +1909-05-05 12:34:56.1532,1909-12-25 +1909-05-05 12:34:56.1533,1909-12-25 +1909-05-05 12:34:56.1534,1909-12-25 +1909-05-05 12:34:56.1535,1909-12-25 +1909-05-05 12:34:56.1536,1909-12-25 +1909-05-05 12:34:56.1537,1909-12-25 +1909-05-05 12:34:56.1538,1909-12-25 +1909-05-05 12:34:56.1539,1909-12-25 +1909-05-05 12:34:56.154,1909-12-25 +1909-05-05 12:34:56.1541,1909-12-25 +1909-05-05 12:34:56.1542,1909-12-25 +1909-05-05 12:34:56.1543,1909-12-25 +1909-05-05 12:34:56.1544,1909-12-25 +1909-05-05 12:34:56.1545,1909-12-25 +1909-05-05 12:34:56.1546,1909-12-25 +1909-05-05 12:34:56.1547,1909-12-25 +1909-05-05 12:34:56.1548,1909-12-25 +1909-05-05 12:34:56.1549,1909-12-25 +1909-05-05 12:34:56.155,1909-12-25 +1909-05-05 12:34:56.1551,1909-12-25 +1909-05-05 12:34:56.1552,1909-12-25 +1909-05-05 12:34:56.1553,1909-12-25 +1909-05-05 12:34:56.1554,1909-12-25 +1909-05-05 12:34:56.1555,1909-12-25 +1909-05-05 12:34:56.1556,1909-12-25 +1909-05-05 12:34:56.1557,1909-12-25 +1909-05-05 12:34:56.1558,1909-12-25 +1909-05-05 12:34:56.1559,1909-12-25 +1909-05-05 12:34:56.156,1909-12-25 +1909-05-05 12:34:56.1561,1909-12-25 +1909-05-05 12:34:56.1562,1909-12-25 +1909-05-05 12:34:56.1563,1909-12-25 +1909-05-05 12:34:56.1564,1909-12-25 +1909-05-05 12:34:56.1565,1909-12-25 +1909-05-05 12:34:56.1566,1909-12-25 +1909-05-05 12:34:56.1567,1909-12-25 +1909-05-05 12:34:56.1568,1909-12-25 +1909-05-05 12:34:56.1569,1909-12-25 +1909-05-05 12:34:56.157,1909-12-25 +1909-05-05 12:34:56.1571,1909-12-25 +1909-05-05 12:34:56.1572,1909-12-25 +1909-05-05 12:34:56.1573,1909-12-25 +1909-05-05 12:34:56.1574,1909-12-25 +1909-05-05 12:34:56.1575,1909-12-25 +1909-05-05 12:34:56.1576,1909-12-25 +1909-05-05 12:34:56.1577,1909-12-25 +1909-05-05 12:34:56.1578,1909-12-25 +1909-05-05 12:34:56.1579,1909-12-25 +1909-05-05 12:34:56.158,1909-12-25 +1909-05-05 12:34:56.1581,1909-12-25 +1909-05-05 12:34:56.1582,1909-12-25 +1909-05-05 12:34:56.1583,1909-12-25 +1909-05-05 12:34:56.1584,1909-12-25 +1909-05-05 12:34:56.1585,1909-12-25 +1909-05-05 12:34:56.1586,1909-12-25 +1909-05-05 12:34:56.1587,1909-12-25 +1909-05-05 12:34:56.1588,1909-12-25 +1909-05-05 12:34:56.1589,1909-12-25 +1909-05-05 12:34:56.159,1909-12-25 +1909-05-05 12:34:56.1591,1909-12-25 +1909-05-05 12:34:56.1592,1909-12-25 +1909-05-05 12:34:56.1593,1909-12-25 +1909-05-05 12:34:56.1594,1909-12-25 +1909-05-05 12:34:56.1595,1909-12-25 +1909-05-05 12:34:56.1596,1909-12-25 +1909-05-05 12:34:56.1597,1909-12-25 +1909-05-05 12:34:56.1598,1909-12-25 +1909-05-05 12:34:56.1599,1909-12-25 +1909-05-05 12:34:56.16,1909-12-25 +1909-05-05 12:34:56.1601,1909-12-25 +1909-05-05 12:34:56.1602,1909-12-25 +1909-05-05 12:34:56.1603,1909-12-25 +1909-05-05 12:34:56.1604,1909-12-25 +1909-05-05 12:34:56.1605,1909-12-25 +1909-05-05 12:34:56.1606,1909-12-25 +1909-05-05 12:34:56.1607,1909-12-25 +1909-05-05 12:34:56.1608,1909-12-25 +1909-05-05 12:34:56.1609,1909-12-25 +1909-05-05 12:34:56.161,1909-12-25 +1909-05-05 12:34:56.1611,1909-12-25 +1909-05-05 12:34:56.1612,1909-12-25 +1909-05-05 12:34:56.1613,1909-12-25 +1909-05-05 12:34:56.1614,1909-12-25 +1909-05-05 12:34:56.1615,1909-12-25 +1909-05-05 12:34:56.1616,1909-12-25 +1909-05-05 12:34:56.1617,1909-12-25 +1909-05-05 12:34:56.1618,1909-12-25 +1909-05-05 12:34:56.1619,1909-12-25 +1909-05-05 12:34:56.162,1909-12-25 +1909-05-05 12:34:56.1621,1909-12-25 +1909-05-05 12:34:56.1622,1909-12-25 +1909-05-05 12:34:56.1623,1909-12-25 +1909-05-05 12:34:56.1624,1909-12-25 +1909-05-05 12:34:56.1625,1909-12-25 +1909-05-05 12:34:56.1626,1909-12-25 +1909-05-05 12:34:56.1627,1909-12-25 +1909-05-05 12:34:56.1628,1909-12-25 +1909-05-05 12:34:56.1629,1909-12-25 +1909-05-05 12:34:56.163,1909-12-25 +1909-05-05 12:34:56.1631,1909-12-25 +1909-05-05 12:34:56.1632,1909-12-25 +1909-05-05 12:34:56.1633,1909-12-25 +1909-05-05 12:34:56.1634,1909-12-25 +1909-05-05 12:34:56.1635,1909-12-25 +1909-05-05 12:34:56.1636,1909-12-25 +1909-05-05 12:34:56.1637,1909-12-25 +1909-05-05 12:34:56.1638,1909-12-25 +1909-05-05 12:34:56.1639,1909-12-25 +1909-05-05 12:34:56.164,1909-12-25 +1909-05-05 12:34:56.1641,1909-12-25 +1909-05-05 12:34:56.1642,1909-12-25 +1909-05-05 12:34:56.1643,1909-12-25 +1909-05-05 12:34:56.1644,1909-12-25 +1909-05-05 12:34:56.1645,1909-12-25 +1909-05-05 12:34:56.1646,1909-12-25 +1909-05-05 12:34:56.1647,1909-12-25 +1909-05-05 12:34:56.1648,1909-12-25 +1909-05-05 12:34:56.1649,1909-12-25 +1909-05-05 12:34:56.165,1909-12-25 +1909-05-05 12:34:56.1651,1909-12-25 +1909-05-05 12:34:56.1652,1909-12-25 +1909-05-05 12:34:56.1653,1909-12-25 +1909-05-05 12:34:56.1654,1909-12-25 +1909-05-05 12:34:56.1655,1909-12-25 +1909-05-05 12:34:56.1656,1909-12-25 +1909-05-05 12:34:56.1657,1909-12-25 +1909-05-05 12:34:56.1658,1909-12-25 +1909-05-05 12:34:56.1659,1909-12-25 +1909-05-05 12:34:56.166,1909-12-25 +1909-05-05 12:34:56.1661,1909-12-25 +1909-05-05 12:34:56.1662,1909-12-25 +1909-05-05 12:34:56.1663,1909-12-25 +1909-05-05 12:34:56.1664,1909-12-25 +1909-05-05 12:34:56.1665,1909-12-25 +1909-05-05 12:34:56.1666,1909-12-25 +1909-05-05 12:34:56.1667,1909-12-25 +1909-05-05 12:34:56.1668,1909-12-25 +1909-05-05 12:34:56.1669,1909-12-25 +1909-05-05 12:34:56.167,1909-12-25 +1909-05-05 12:34:56.1671,1909-12-25 +1909-05-05 12:34:56.1672,1909-12-25 +1909-05-05 12:34:56.1673,1909-12-25 +1909-05-05 12:34:56.1674,1909-12-25 +1909-05-05 12:34:56.1675,1909-12-25 +1909-05-05 12:34:56.1676,1909-12-25 +1909-05-05 12:34:56.1677,1909-12-25 +1909-05-05 12:34:56.1678,1909-12-25 +1909-05-05 12:34:56.1679,1909-12-25 +1909-05-05 12:34:56.168,1909-12-25 +1909-05-05 12:34:56.1681,1909-12-25 +1909-05-05 12:34:56.1682,1909-12-25 +1909-05-05 12:34:56.1683,1909-12-25 +1909-05-05 12:34:56.1684,1909-12-25 +1909-05-05 12:34:56.1685,1909-12-25 +1909-05-05 12:34:56.1686,1909-12-25 +1909-05-05 12:34:56.1687,1909-12-25 +1909-05-05 12:34:56.1688,1909-12-25 +1909-05-05 12:34:56.1689,1909-12-25 +1909-05-05 12:34:56.169,1909-12-25 +1909-05-05 12:34:56.1691,1909-12-25 +1909-05-05 12:34:56.1692,1909-12-25 +1909-05-05 12:34:56.1693,1909-12-25 +1909-05-05 12:34:56.1694,1909-12-25 +1909-05-05 12:34:56.1695,1909-12-25 +1909-05-05 12:34:56.1696,1909-12-25 +1909-05-05 12:34:56.1697,1909-12-25 +1909-05-05 12:34:56.1698,1909-12-25 +1909-05-05 12:34:56.1699,1909-12-25 +1909-05-05 12:34:56.17,1909-12-25 +1909-05-05 12:34:56.1701,1909-12-25 +1909-05-05 12:34:56.1702,1909-12-25 +1909-05-05 12:34:56.1703,1909-12-25 +1909-05-05 12:34:56.1704,1909-12-25 +1909-05-05 12:34:56.1705,1909-12-25 +1909-05-05 12:34:56.1706,1909-12-25 +1909-05-05 12:34:56.1707,1909-12-25 +1909-05-05 12:34:56.1708,1909-12-25 +1909-05-05 12:34:56.1709,1909-12-25 +1909-05-05 12:34:56.171,1909-12-25 +1909-05-05 12:34:56.1711,1909-12-25 +1909-05-05 12:34:56.1712,1909-12-25 +1909-05-05 12:34:56.1713,1909-12-25 +1909-05-05 12:34:56.1714,1909-12-25 +1909-05-05 12:34:56.1715,1909-12-25 +1909-05-05 12:34:56.1716,1909-12-25 +1909-05-05 12:34:56.1717,1909-12-25 +1909-05-05 12:34:56.1718,1909-12-25 +1909-05-05 12:34:56.1719,1909-12-25 +1909-05-05 12:34:56.172,1909-12-25 +1909-05-05 12:34:56.1721,1909-12-25 +1909-05-05 12:34:56.1722,1909-12-25 +1909-05-05 12:34:56.1723,1909-12-25 +1909-05-05 12:34:56.1724,1909-12-25 +1909-05-05 12:34:56.1725,1909-12-25 +1909-05-05 12:34:56.1726,1909-12-25 +1909-05-05 12:34:56.1727,1909-12-25 +1909-05-05 12:34:56.1728,1909-12-25 +1909-05-05 12:34:56.1729,1909-12-25 +1909-05-05 12:34:56.173,1909-12-25 +1909-05-05 12:34:56.1731,1909-12-25 +1909-05-05 12:34:56.1732,1909-12-25 +1909-05-05 12:34:56.1733,1909-12-25 +1909-05-05 12:34:56.1734,1909-12-25 +1909-05-05 12:34:56.1735,1909-12-25 +1909-05-05 12:34:56.1736,1909-12-25 +1909-05-05 12:34:56.1737,1909-12-25 +1909-05-05 12:34:56.1738,1909-12-25 +1909-05-05 12:34:56.1739,1909-12-25 +1909-05-05 12:34:56.174,1909-12-25 +1909-05-05 12:34:56.1741,1909-12-25 +1909-05-05 12:34:56.1742,1909-12-25 +1909-05-05 12:34:56.1743,1909-12-25 +1909-05-05 12:34:56.1744,1909-12-25 +1909-05-05 12:34:56.1745,1909-12-25 +1909-05-05 12:34:56.1746,1909-12-25 +1909-05-05 12:34:56.1747,1909-12-25 +1909-05-05 12:34:56.1748,1909-12-25 +1909-05-05 12:34:56.1749,1909-12-25 +1909-05-05 12:34:56.175,1909-12-25 +1909-05-05 12:34:56.1751,1909-12-25 +1909-05-05 12:34:56.1752,1909-12-25 +1909-05-05 12:34:56.1753,1909-12-25 +1909-05-05 12:34:56.1754,1909-12-25 +1909-05-05 12:34:56.1755,1909-12-25 +1909-05-05 12:34:56.1756,1909-12-25 +1909-05-05 12:34:56.1757,1909-12-25 +1909-05-05 12:34:56.1758,1909-12-25 +1909-05-05 12:34:56.1759,1909-12-25 +1909-05-05 12:34:56.176,1909-12-25 +1909-05-05 12:34:56.1761,1909-12-25 +1909-05-05 12:34:56.1762,1909-12-25 +1909-05-05 12:34:56.1763,1909-12-25 +1909-05-05 12:34:56.1764,1909-12-25 +1909-05-05 12:34:56.1765,1909-12-25 +1909-05-05 12:34:56.1766,1909-12-25 +1909-05-05 12:34:56.1767,1909-12-25 +1909-05-05 12:34:56.1768,1909-12-25 +1909-05-05 12:34:56.1769,1909-12-25 +1909-05-05 12:34:56.177,1909-12-25 +1909-05-05 12:34:56.1771,1909-12-25 +1909-05-05 12:34:56.1772,1909-12-25 +1909-05-05 12:34:56.1773,1909-12-25 +1909-05-05 12:34:56.1774,1909-12-25 +1909-05-05 12:34:56.1775,1909-12-25 +1909-05-05 12:34:56.1776,1909-12-25 +1909-05-05 12:34:56.1777,1909-12-25 +1909-05-05 12:34:56.1778,1909-12-25 +1909-05-05 12:34:56.1779,1909-12-25 +1909-05-05 12:34:56.178,1909-12-25 +1909-05-05 12:34:56.1781,1909-12-25 +1909-05-05 12:34:56.1782,1909-12-25 +1909-05-05 12:34:56.1783,1909-12-25 +1909-05-05 12:34:56.1784,1909-12-25 +1909-05-05 12:34:56.1785,1909-12-25 +1909-05-05 12:34:56.1786,1909-12-25 +1909-05-05 12:34:56.1787,1909-12-25 +1909-05-05 12:34:56.1788,1909-12-25 +1909-05-05 12:34:56.1789,1909-12-25 +1909-05-05 12:34:56.179,1909-12-25 +1909-05-05 12:34:56.1791,1909-12-25 +1909-05-05 12:34:56.1792,1909-12-25 +1909-05-05 12:34:56.1793,1909-12-25 +1909-05-05 12:34:56.1794,1909-12-25 +1909-05-05 12:34:56.1795,1909-12-25 +1909-05-05 12:34:56.1796,1909-12-25 +1909-05-05 12:34:56.1797,1909-12-25 +1909-05-05 12:34:56.1798,1909-12-25 +1909-05-05 12:34:56.1799,1909-12-25 +1909-05-05 12:34:56.18,1909-12-25 +1909-05-05 12:34:56.1801,1909-12-25 +1909-05-05 12:34:56.1802,1909-12-25 +1909-05-05 12:34:56.1803,1909-12-25 +1909-05-05 12:34:56.1804,1909-12-25 +1909-05-05 12:34:56.1805,1909-12-25 +1909-05-05 12:34:56.1806,1909-12-25 +1909-05-05 12:34:56.1807,1909-12-25 +1909-05-05 12:34:56.1808,1909-12-25 +1909-05-05 12:34:56.1809,1909-12-25 +1909-05-05 12:34:56.181,1909-12-25 +1909-05-05 12:34:56.1811,1909-12-25 +1909-05-05 12:34:56.1812,1909-12-25 +1909-05-05 12:34:56.1813,1909-12-25 +1909-05-05 12:34:56.1814,1909-12-25 +1909-05-05 12:34:56.1815,1909-12-25 +1909-05-05 12:34:56.1816,1909-12-25 +1909-05-05 12:34:56.1817,1909-12-25 +1909-05-05 12:34:56.1818,1909-12-25 +1909-05-05 12:34:56.1819,1909-12-25 +1909-05-05 12:34:56.182,1909-12-25 +1909-05-05 12:34:56.1821,1909-12-25 +1909-05-05 12:34:56.1822,1909-12-25 +1909-05-05 12:34:56.1823,1909-12-25 +1909-05-05 12:34:56.1824,1909-12-25 +1909-05-05 12:34:56.1825,1909-12-25 +1909-05-05 12:34:56.1826,1909-12-25 +1909-05-05 12:34:56.1827,1909-12-25 +1909-05-05 12:34:56.1828,1909-12-25 +1909-05-05 12:34:56.1829,1909-12-25 +1909-05-05 12:34:56.183,1909-12-25 +1909-05-05 12:34:56.1831,1909-12-25 +1909-05-05 12:34:56.1832,1909-12-25 +1909-05-05 12:34:56.1833,1909-12-25 +1909-05-05 12:34:56.1834,1909-12-25 +1909-05-05 12:34:56.1835,1909-12-25 +1909-05-05 12:34:56.1836,1909-12-25 +1909-05-05 12:34:56.1837,1909-12-25 +1909-05-05 12:34:56.1838,1909-12-25 +1909-05-05 12:34:56.1839,1909-12-25 +1909-05-05 12:34:56.184,1909-12-25 +1909-05-05 12:34:56.1841,1909-12-25 +1909-05-05 12:34:56.1842,1909-12-25 +1909-05-05 12:34:56.1843,1909-12-25 +1909-05-05 12:34:56.1844,1909-12-25 +1909-05-05 12:34:56.1845,1909-12-25 +1909-05-05 12:34:56.1846,1909-12-25 +1909-05-05 12:34:56.1847,1909-12-25 +1909-05-05 12:34:56.1848,1909-12-25 +1909-05-05 12:34:56.1849,1909-12-25 +1909-05-05 12:34:56.185,1909-12-25 +1909-05-05 12:34:56.1851,1909-12-25 +1909-05-05 12:34:56.1852,1909-12-25 +1909-05-05 12:34:56.1853,1909-12-25 +1909-05-05 12:34:56.1854,1909-12-25 +1909-05-05 12:34:56.1855,1909-12-25 +1909-05-05 12:34:56.1856,1909-12-25 +1909-05-05 12:34:56.1857,1909-12-25 +1909-05-05 12:34:56.1858,1909-12-25 +1909-05-05 12:34:56.1859,1909-12-25 +1909-05-05 12:34:56.186,1909-12-25 +1909-05-05 12:34:56.1861,1909-12-25 +1909-05-05 12:34:56.1862,1909-12-25 +1909-05-05 12:34:56.1863,1909-12-25 +1909-05-05 12:34:56.1864,1909-12-25 +1909-05-05 12:34:56.1865,1909-12-25 +1909-05-05 12:34:56.1866,1909-12-25 +1909-05-05 12:34:56.1867,1909-12-25 +1909-05-05 12:34:56.1868,1909-12-25 +1909-05-05 12:34:56.1869,1909-12-25 +1909-05-05 12:34:56.187,1909-12-25 +1909-05-05 12:34:56.1871,1909-12-25 +1909-05-05 12:34:56.1872,1909-12-25 +1909-05-05 12:34:56.1873,1909-12-25 +1909-05-05 12:34:56.1874,1909-12-25 +1909-05-05 12:34:56.1875,1909-12-25 +1909-05-05 12:34:56.1876,1909-12-25 +1909-05-05 12:34:56.1877,1909-12-25 +1909-05-05 12:34:56.1878,1909-12-25 +1909-05-05 12:34:56.1879,1909-12-25 +1909-05-05 12:34:56.188,1909-12-25 +1909-05-05 12:34:56.1881,1909-12-25 +1909-05-05 12:34:56.1882,1909-12-25 +1909-05-05 12:34:56.1883,1909-12-25 +1909-05-05 12:34:56.1884,1909-12-25 +1909-05-05 12:34:56.1885,1909-12-25 +1909-05-05 12:34:56.1886,1909-12-25 +1909-05-05 12:34:56.1887,1909-12-25 +1909-05-05 12:34:56.1888,1909-12-25 +1909-05-05 12:34:56.1889,1909-12-25 +1909-05-05 12:34:56.189,1909-12-25 +1909-05-05 12:34:56.1891,1909-12-25 +1909-05-05 12:34:56.1892,1909-12-25 +1909-05-05 12:34:56.1893,1909-12-25 +1909-05-05 12:34:56.1894,1909-12-25 +1909-05-05 12:34:56.1895,1909-12-25 +1909-05-05 12:34:56.1896,1909-12-25 +1909-05-05 12:34:56.1897,1909-12-25 +1909-05-05 12:34:56.1898,1909-12-25 +1909-05-05 12:34:56.1899,1909-12-25 +1909-05-05 12:34:56.19,1909-12-25 +1909-05-05 12:34:56.1901,1909-12-25 +1909-05-05 12:34:56.1902,1909-12-25 +1909-05-05 12:34:56.1903,1909-12-25 +1909-05-05 12:34:56.1904,1909-12-25 +1909-05-05 12:34:56.1905,1909-12-25 +1909-05-05 12:34:56.1906,1909-12-25 +1909-05-05 12:34:56.1907,1909-12-25 +1909-05-05 12:34:56.1908,1909-12-25 +1909-05-05 12:34:56.1909,1909-12-25 +1909-05-05 12:34:56.191,1909-12-25 +1909-05-05 12:34:56.1911,1909-12-25 +1909-05-05 12:34:56.1912,1909-12-25 +1909-05-05 12:34:56.1913,1909-12-25 +1909-05-05 12:34:56.1914,1909-12-25 +1909-05-05 12:34:56.1915,1909-12-25 +1909-05-05 12:34:56.1916,1909-12-25 +1909-05-05 12:34:56.1917,1909-12-25 +1909-05-05 12:34:56.1918,1909-12-25 +1909-05-05 12:34:56.1919,1909-12-25 +1909-05-05 12:34:56.192,1909-12-25 +1909-05-05 12:34:56.1921,1909-12-25 +1909-05-05 12:34:56.1922,1909-12-25 +1909-05-05 12:34:56.1923,1909-12-25 +1909-05-05 12:34:56.1924,1909-12-25 +1909-05-05 12:34:56.1925,1909-12-25 +1909-05-05 12:34:56.1926,1909-12-25 +1909-05-05 12:34:56.1927,1909-12-25 +1909-05-05 12:34:56.1928,1909-12-25 +1909-05-05 12:34:56.1929,1909-12-25 +1909-05-05 12:34:56.193,1909-12-25 +1909-05-05 12:34:56.1931,1909-12-25 +1909-05-05 12:34:56.1932,1909-12-25 +1909-05-05 12:34:56.1933,1909-12-25 +1909-05-05 12:34:56.1934,1909-12-25 +1909-05-05 12:34:56.1935,1909-12-25 +1909-05-05 12:34:56.1936,1909-12-25 +1909-05-05 12:34:56.1937,1909-12-25 +1909-05-05 12:34:56.1938,1909-12-25 +1909-05-05 12:34:56.1939,1909-12-25 +1909-05-05 12:34:56.194,1909-12-25 +1909-05-05 12:34:56.1941,1909-12-25 +1909-05-05 12:34:56.1942,1909-12-25 +1909-05-05 12:34:56.1943,1909-12-25 +1909-05-05 12:34:56.1944,1909-12-25 +1909-05-05 12:34:56.1945,1909-12-25 +1909-05-05 12:34:56.1946,1909-12-25 +1909-05-05 12:34:56.1947,1909-12-25 +1909-05-05 12:34:56.1948,1909-12-25 +1909-05-05 12:34:56.1949,1909-12-25 +1909-05-05 12:34:56.195,1909-12-25 +1909-05-05 12:34:56.1951,1909-12-25 +1909-05-05 12:34:56.1952,1909-12-25 +1909-05-05 12:34:56.1953,1909-12-25 +1909-05-05 12:34:56.1954,1909-12-25 +1909-05-05 12:34:56.1955,1909-12-25 +1909-05-05 12:34:56.1956,1909-12-25 +1909-05-05 12:34:56.1957,1909-12-25 +1909-05-05 12:34:56.1958,1909-12-25 +1909-05-05 12:34:56.1959,1909-12-25 +1909-05-05 12:34:56.196,1909-12-25 +1909-05-05 12:34:56.1961,1909-12-25 +1909-05-05 12:34:56.1962,1909-12-25 +1909-05-05 12:34:56.1963,1909-12-25 +1909-05-05 12:34:56.1964,1909-12-25 +1909-05-05 12:34:56.1965,1909-12-25 +1909-05-05 12:34:56.1966,1909-12-25 +1909-05-05 12:34:56.1967,1909-12-25 +1909-05-05 12:34:56.1968,1909-12-25 +1909-05-05 12:34:56.1969,1909-12-25 +1909-05-05 12:34:56.197,1909-12-25 +1909-05-05 12:34:56.1971,1909-12-25 +1909-05-05 12:34:56.1972,1909-12-25 +1909-05-05 12:34:56.1973,1909-12-25 +1909-05-05 12:34:56.1974,1909-12-25 +1909-05-05 12:34:56.1975,1909-12-25 +1909-05-05 12:34:56.1976,1909-12-25 +1909-05-05 12:34:56.1977,1909-12-25 +1909-05-05 12:34:56.1978,1909-12-25 +1909-05-05 12:34:56.1979,1909-12-25 +1909-05-05 12:34:56.198,1909-12-25 +1909-05-05 12:34:56.1981,1909-12-25 +1909-05-05 12:34:56.1982,1909-12-25 +1909-05-05 12:34:56.1983,1909-12-25 +1909-05-05 12:34:56.1984,1909-12-25 +1909-05-05 12:34:56.1985,1909-12-25 +1909-05-05 12:34:56.1986,1909-12-25 +1909-05-05 12:34:56.1987,1909-12-25 +1909-05-05 12:34:56.1988,1909-12-25 +1909-05-05 12:34:56.1989,1909-12-25 +1909-05-05 12:34:56.199,1909-12-25 +1909-05-05 12:34:56.1991,1909-12-25 +1909-05-05 12:34:56.1992,1909-12-25 +1909-05-05 12:34:56.1993,1909-12-25 +1909-05-05 12:34:56.1994,1909-12-25 +1909-05-05 12:34:56.1995,1909-12-25 +1909-05-05 12:34:56.1996,1909-12-25 +1909-05-05 12:34:56.1997,1909-12-25 +1909-05-05 12:34:56.1998,1909-12-25 +1909-05-05 12:34:56.1999,1909-12-25 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc new file mode 100644 index 0000000000000000000000000000000000000000..f51ffdbd03a43fadbedce302ffa8e5967a30ad59 GIT binary patch literal 30941 zcmeI51yq!6yYFY1p=;6DUCQt4DAB&1V9K%}IV7?2VLBo!ov29;7` zK;jJWhPdA^&v(|^XK&V0x!3*F%`9hK*Zu$fuj|p&kz)n`u+RxnF=92e&IS)ywE$57 zz+C*2r4<0gpdEwhZ|Y0^$;CI|xXKhhSJ^aOw}{T`T6|~omgRWf_uxH&b-{8>HEA1~ z&-69|wFhE}mp3-YR-3*LqL2e@k#{G>0s#O3LVV;S#4#{7BS4VXqbSISQDC9v17D&n zgOG4rb^e(|N#XkWa*vfFZn=GcBF8k-doL zdUxG5y@)$fL5*g@Y-!l5WNh)8$2NB;-b9edZ|fjsv@i|6|Hacd*q)BT9gf!s#8N^o zFTR2t^E=_!A-6rF;MF-&S@bm+)o%-ojN|2L-Mvko47YE_djj%;<7!FE5*~$xnnBM# zvD4-CfD`Gb*t$Q*$iW!>iNGkBX#@~*G;>r!;#^ETPG50A)`8U;Kx930LipKgJ&8EN z4}Ix}b2i@4z^sYWO?3>KdXUR+A0nqhTO;_DEoBn?-d)^39*F#SJCstA;l=B$)66L` z-y!0z=u-gHlaEl@AS(1OCe%6E0<+B2jF|PQv7zlc-e9srY_v`W?`8(T~E+{q`PMBnLfU%6#gJ=NC}oggCw= zLp@Luj++C(jDujty{EujCFD&0Y!rNB0;3go8R!B)ReBJL{L`momkbxoS~Ll^C&T=& zNBEvAAnyWL9lKmLHwr}GkeJJhe9V43g@1Hz*>4MakB;JQYLb}R3bRU_&O9-@yvE57 z`$+1iHh9-kUK20c7%uXLiK(T2@(=rzYRz9P)v8gHj zn?X&<+%uz0y~EFqlzPb{^&9#_pKH@@zy`m@+Zc4`%6AfazFz#A#Pu=0^dpTCk7*^Y zkGauKx98W0)iD7HPSYwz9~;)4&OU0mo%0*M$VX!B7BQ|=+)K5_t%=4ph|rP-$q%mR zWeYLJU77X%o9rU4WX|(t6YZ1rspwal5K*??-Hf!~L#H>yS{5SjOMNG4nRE2-7dZ%W z*<+mMYKHq9y~95UquaYIwJC!9rnA3q(=i9L(F3use#F?Dz5E2C@`TV+mg{Snf+)u7 ztm6|R4OyOyFqwLcJXeyZ9aI+KfI(MG3Jiu|^7sxB-up%dFhV_7bjSMpy|Y}mW(Yht zz&_sLphnp3O|A?$Zr>${{cU#sOT+`S5^)cx_{i!k8ImlS*V^}Icq|!d+x4zI%=9Mo z+2q8Fq-)o1e3;-(WQ(a(UHv$vLjw7~>(Zb?PeJK*rGo2RQ>r!9$&@SfHG|rp^Am=7 z{2!yOT#6plP0LRgd@KEwOzKS(jt!GlYB!Zwa^0Ou#Er7~Oyjpc4RQE)F4HYIZ$?hM zwY?k5^Nr1HLAm*jKn}6t4UQ-jl6>H$&9Bj%3dt~hCj-I*qD6b?| z@w1WHTrcYZLv?PF6ji^xPC@case=QUn^s~}qorD-5ztDd1ei;aa3XT znfkbTx<}5?zK$)dOMI|WV=FrAi&4{i)h3ke<(lRXREIfr``YVU2~$!B>U3L?(9imQ zgD(3~fZSHw2QmM zR__dD;jdJNiQIH{?rr1j?U_ieX%L~>7`@e-HW_{6m>fIO$sS_7h zXD}(s^o{wIv2lZjni{N9xti zpsBI$PT@)sPiL3aHttod#yUqI7m=M-=heK)G1gb;QsW_hm5M(cGZ@P5Q;@yv z?%=*H;;i8ZCv{%YTk9*GZ2DA>URMBhm)M=(SsSC7h?ID#@0AO^V~g)yUAIM!f3N#B z?USoKyS1h>iKYvyjpLtuaBaLxy|%>H=eMWEs2U?hbecu(inw$FhP)#vaQJfh)i%(U zyd$xVXR+2@)tlOIp>R^V+xIo=yKJv$m2W;M_ihup@&i_QSkQmvU`P?V6VbBi7Vw44 ze?@8AP01Iy4fPs0Xfp|(4nBUB)h=-SC zNWKP(e?(Vz#TbW^yAPAe&}D=$)Pt>Dai79T-wh`-&~VGKgfd{=R%%mExn>q7ZxF6q zkNw4!PI!jlR&n`j?kiz_-gx&n2xDf1GKxfIsDmEFBoPp`V#teL!asia&5nvhn?iSh z`&pjfu$D`nu(*PW`@{WZYTN;4%3_y1_LuT*IF{n3ooYB%SN+J4-`=H7VeVP}vRM=3 zYtvooz7IYj$h|h%*XG==Xi2qO`9>EaNc%CBmaS8HhhKr2MH$|(yh(7!=TRhash<8s|ToLA?J<8Nu;VvEW z8_&OAt^UYf@i2a8MT}-pRs87(cNLeF10{kqPE0v^88DHoFjK%mAaOnu$yi+4S7kZ8 ze7}QO_xY>$?y!IwV&jb8Y#|6z1!4!r_^tR;p5WNdYrp$;E1l#$;^rn(om&RTk;dP&WF?<4|WHJ8S*v zw0cYWQ5WA~L9UeK$=F}y@R8NNs{FAFn&`C8ep4axpY@u;mf@}Bheun!>npoi{_86( z(^*G4LuMTNL*M;;T3pt5R}bqxO$9jp&8b7pmgALOsl$Ndp#YbsE!VdSmwsgX9}gXy z9d8{2)=om6MAesJ3N*OnT!Bf1vXwaBsLe{VyV*;g2L8-twZeqS2iL`m)i`8qbRw>^ zBr4g652R|iKPAdsaTm`%8C1PC`}hu)p?tvFyyP>z%V(6ZeKWJX?u(^KS;@AuiPV8j zVtH+erEb}?5og`QUWQF<4 z!p-w7>8&?zi=w6(Ufly1uogR&^(V1effr)r4QaZN>uc>E*jr*J$f-g5XY(D%Y z24bF%f!@6WSLO*G-EDJ0WK4hScx|@#aC%dyWr^wtR(BA$PMW5eQTb*SfN;(JWiCuDtX^v0pu^;ko* zj(;x-8hvbI{0hh1*}y^f4w{>l5N)N(kf$G<)v>gRa9_D$Iux_&S5ysltv-3q}Gcj32C{eK|*Xl4N>eP5MjtrVQ zAGZ=(KoW#vPQWEGQ9F7O_tXQsePYqM+ze>g10Txx?+UV7VD-{6?Z7Y4;8z z-Mk*OXL44*A-Vq>z1h5Wb5DYAf~_=r^^5D|s2obISlYrWcvIJ~3a)e56b|>8+CL5% zgdUizJWd$AY-^*%-BVJYUK&mKK_%avjRI}*0&NP3JmDP$E9xjxUmN$%H_OpgeB@+S zf@PiHX5pJeDb)>v6hvv=?JU6Ql!gW&>?*;_ZnmR~Q@3DsHyvqv`>v!m-4#avZW)80 zot|)4z!uw}x~7^O+YrOog}~8jVRV$^`QWqXnYtl#&pX8pNA(L{p`kk0Y6vHWTu3%eXAs_<3b~Bv1PP-;K#OQ%m79a@x?o?ceP-TJkXh!Gm_&%%ZH1-JYJI(_ zbVn%?JT4E0;2|rI?r9_IogS&d(*2E37=lsdb#qfEt^h8?86RM zQM_S!O$A-v@8xKBFqrQ()NNyxZ*jORjZ95Bvb}a}%KLDawS2d43U1OE_pRy8tGfbd zDV?hC%pVXWS_F%Tb{QGN3H812JARg%fPE;o=5<4OL46n#1aU^ za+MBNk468KlLN(lW==NT3bhNCdym$YlLy7#hg0LUGLwfm@}kz6KW)ZXai!pAz|npn zN9f#^vCq%h#{ay{#Nhsq3QT+pRX-BX8TPNQ-9$r^U2)21uKLmR5jyrRlZ3)Ju3#p%;Y2r-)EP>)KGe% z{LMO@SUL6jaC*MRj&cHZys#LH$sLt@pPl%blC9Os-aMX&7ei%NKg{4~SJQrYE#7(X z%H-3I*8zhZlLa0xHPg3ZYghPIi_C_gl7({jl)JXlG*-AW>DXlxm@P1v9+=B^htEG+ zj|+aBe@Q|oLDIfEsQS@*LQIB|W}h`Z8GrocR9Y!|H#B~6qnm1I0*_~+t~|VvcW3eVLHV68jeog9ita$q2tx#3CqpknkKHY=-m^G5J#;L!yLF86?H@)y z){oIj%sQ#p#ZRMNyh1*IriQI}W}kbm05UGJiVsDM&G6j1Y2L?lL^S=qY5m8t?KRK0 zi$VrNUl!7CycW9mZq&Nz#?qJ7mj=mS9GZ+i;0N4TC|P}GkXYhSV>D87`0NIGiKmCL zN6FALvrPThpI=at9XS|I>Iq;Qq?x?H!M;)5ZP4cYF=O%ts?iIQS5{v-e%yLZ`!PxK z1&xoT&(1r%Nc@j+O)rRitZeaI1?%QhI_^`?@+#KjCc7Dx!Bxr(ecMMSvK;F{AMPlY z&1o37AVR6?u|@9aJI^{#yaQ#K`u3cUgk8QJ>xi`J@3;YQqoZ!GNPXY_?7g6t4aR^4 z)}zk;NFK1E0ag@pYi}NeLB(L^%T6rIe!WejD6Hu{2Ko~dUuO3<<>jy{UJNn|;>(t(0pPRo=Kp-k~&CT4}Rp06eb7zSykZAypVx z4UHd0VaQi|GZNlrT%6V8Y}a|-o@pR;pnI!5d5=P9(X zN;ydsg0OTdWFJSzhj=}fWlvNYW3PFkPq~m&jURALd?A0U=LW^&NPZ4|8LBi|}>nM(3Q-MuHt z_fVmOj(+7L`!X)uc~dK_O#z5k&}A~e;qv|b=`9N2)ey~dk+5IuTqGnpzaO+XzaJd@ zWk1M&em`hNRfVZGZk!f{scvnV5a{x5Vj?ZFuZ0gO|eQq^QKEyZ);=9 z-0P+D`#~2@OQ+`(I(h{K?B965|>Yr58 zfl?pO?FJ9*KI}|6e+`)&SA12a>51?E0{w^Oqn(kguj!H!uqUnkcyrR|Vjmg$I^SfL zUJEqK=ou5Msq&&+f>-wqtiZ}O)-y6&8$|QmJXn?%-{Bv-+Iy|=bGyeynjpU_8TY!l zCI(93^c zO+BCP*3|7%_G}ogFMCYA`O;J+;!Wm~@9Xl-c`6F{bdykk-14eT3Q`5xBqghFCz+Z_ zyGJL22sH9-r-9DUW^YRS1uFV>5H-(?p+i@B{1#`jq2DSs52CYw7`1Guw&X&;IR>nY z9JRU}=1uRhwiH4)(T`^z9gSokq)VaG=M-!H5(lx*$3aun7L&mLWF##AB@&7|R?lwX4ZIbobw0&;pS-FYMuKGyL z+jQA;LHkhJImas14%vaD}IpZF_j1rx>JioY_(BCo?jvTdcq`XgT%PdS}Bfm@7O za{JnUG!n+1kA(5(BH=XSzY+;Io%biVrqxb^+t1POw|!s#ii?XzK=E0`I*oH#L(@|k zzi;y-3#Z-on|zMHzuc7?|7|ZgwLbadFQ)u7j}LwJKixdpMXsWYp2j}^>7ju2tfSgg zWF(b3+C84jIx@Qp0HAS6W=Q5sK9{VPY?N%5d?z_9Ie{i5`BidD@&F2k;y{U^)KDfU zCz>Sm3iK*e4SF4F47G;dg1SR}Ws%zLFjD*m5X);1Fg&O3=#kN>e+bEd^xe2iLxlDh55A1LBaJCo^Ey6$(HpLwPcCNy;^q5Cx+i zE3_|rW*t;0#t;}-BE7uVhfl343jpc|Y3=pkQ*)dRN9NIYgicQ=((^a}KyZ59OGSTk z2?j94l?0=7eJ`jt3WD$o;u!{02l-8S?SvD7g#O_-cs| zX;?|1-Q+5W5xEc};6QE12Jjw|k66x`6Pd?C7ElJF2(!%7l-r^rGeZcv16on6D^Msd zSHmu;5fY__rihXUl3x{2z8!}41)Yh-_NfgE2tf1`bdf`j>9Ej{h~$KlkRU0E3LZvX z{a{;_HDt4PVR!AC7J%=+V3dMt6v%cUl9jNX=;}?l!RAOXL5@<~93wOl_EEJfXHSJMrTo7%_Z_I(05;Nd%Zu@9nS)d>} z7dr-L3x;#T7LZc^fKt70aGlR9zu={76z6e?>mIn}!)Aql@Q8qf_#cF&jYd@zr0^4v z5WlM2ONKKjq=f;-490X-Cy58E>qIXDm|`EZLx#;?%5VuvSzsRVc?G8LYB-zTe8oc6 zg7}W8r;l-P`YhyUoWjCrA(HWEorSw~`wqeR12ffN zx_)u)){zy;KMo+s2>pSB0Kg~ax8wqMKd|vpq9KM_TbJBgGJG!eXzgY5&01D`J(bdS)8;)rM=zr3!WvuiZ+(Z_Jf3N9aOiJkVHucqCSlUf68eynO#9IH}n*cj(v&m%M_ z<4}+{tjt<$)|mJUa({asxy%28+)Ms0a$jfv1-U={1-W1U&ms4?Q9Ja~kr>0m%xY5! zM-ak*s-{HPUc_lMNaKrYCNO(hsY`r7RBo-;Ae??;X?+F`@kb=zd#LzLeHd1i;H^dbJq_m>UnMu!>|AMSNkWIE70(pKF z0>1p}dP}u?@Z)m#&~7@psRd`(hekL{>}Sf!4&|eVPcn&4rID%r16VQBsWjq;ltx~j zsU$)l4q#6hPiH-n4`5}ir^9WhKV%fAn7tqeg;TNS-q-J-FQ{jCzP5m_rz#7dM)ju! z036wz$iu9|V697LR4L>D?>`g}_{@m{!huvkWYbO+5VJQpCklvw^@#%Vi9GB?0ZG9O zEvT0b*#;3N2n_Kj0Dl)iPRyp#KbuYQJb0int2ACmg6NpOjiZh0FZBVS^ZLiBS2X$pN!xplWXT%#~<#X3_Y~mDupj6G@}G zovw1k_2dugq~;SKof>gHXNanBwK22ylvORVn9bZo3uibYZaJP?*_+WU@vSi-DY>It z>REEH#!Lv?YNRZt#M#4eyX3orfr;K+o0)paCx@Fr)hg8_3x-Hi3e}Ie=~YIx=Bl-p zUbmXOJA}UmIgK%Xs${LT?f+VVh@~03EP*sGqsA)7JL{q&CR>=gSY;rxABI!=$lQg9vg0!JH5a#`#E|Fj=-Rm5-P; z6=ak%^wp8&p&Fu?O|GP9Gnb;;bW=#pum!-e;i}!z7zFLHoXtUE6KyC-6~Ph)uO)Ah zPPdWLTA*uI+_x2lVN;V$w_;AgIZ@(BlP@vK5^yz>Mbs0x-M&m}$*wo6@tp4HlJKm| zt&Y5p5v-MD{j(L`j4`vi8Au&lRXVqzpq+C{MX8N2PFRQAsX&&az+qG4SwYjV5_)lB zj1FsNzW-*Ppc0w=g)w$2!H2Gj%Te6lmkA{MgVh zxTc{}o{=ciz?OGMmt`Y}zeKuVq@brrOP)gy6Eb2v`7F%vS>nrUjvnR0FfF6)BG1pt zt0Pt7MPwe}X7OSXkI_H{N7d+ooN(!<1WtJi<9e5?bxNu;^+NOw^JuE|Bpfn52k)&u z$A!x|4VR7QPgdVR7cb@UK=aLKr1^4ndby!_1iNYY@t8|n#k^*8;Pdl@5hGhifXnlNmxc)* zRX3?VRyS2sY>b|FWMrOqWFQ7;zeY-q>-)TL+4%~%k0vD+1)`ZZzb{I~>Q$=&^!Mbj zT^jD2vd;(^5?3q)wTL#kd7>@NM)i%PW~2*99ZVKX7QC&0FVF26@#?nQXURHO)w(-g z&Z}#aos#uj+s&UESnIs{*WjiT15Gc9wu2;m{zHBY66XzI9pZt6Z<{0(iDDp+5D!^> zUsvJ#j+^bjdoATRZ*}60+O6}SN%oib`}+6&7};oQ01z=e5S&hrUn?WyM|!=LyC}r( zJ@|eOZj6j#6W)I`Mf#OO{a^Yq)UR}Y@P6pBnGqPOk%TGNq<)@4aTh1}4Km%&SC3In z&WO+2=c9;MXM9Blz)Cf0b0?r)(YUNdHxq^U&n#q`|{%2sO{3N zZtgL}KXNCW8YWIi)vJ{N>NAtY$=GY?SoFj;cziSP`>7|fsqom;=MS&Oi6?OvX|Pyb z;SoBsGS=<>*zup93V5}8_(Aw@P91NZxe$@g#N88K6|moR>Qp=~57-X?up{%OzW`5- zze62z)R}xwO4c5{V2ofQQQ4=ip>>fP^wEil@=R_AT}~S$`up0LKyWTr_x9W3EfXYb z(nhsznggYuP5O7Dk48YG>7NDq)@}ybuz(>CIOxs1G^Bn_V|xBfV_MoVPSTi3Uug=3 z9JO2s5jqGW5L7{|Z&%?jZF%RW;Z6R3g5wnJw|!NDY$m~aibVg4<9wI`LW)a%&trb8 zc}W2t-lu26kQK*Oh3phr0Z9Ro3OOk`xli#zKyBoQU5!BJHOafHsm(9Ne<^G^Sba2| z#k`uTH$?hpT5?+1asxcyle$=DN1-Nm2y}5En)Gt`)Omhn?hF989YTe?QmCF1zWf zs&!MQ{_iuM4?<89YrqA(xu_Jms1&)V6uGDrxu_KRUt1}10dFEN;LYEPf*0@xSugpo z`C~8O%|BNuashAt8&-;3z?*Z~Lok9Nt|%o@a4?U$M`>X8n6TQjq`kCHP{5$kUD(!fU zj|lN`0|5sg0?__~-t-m#A#*WE)#K+*ALM`_yI>1!PDb;wFp_85%-$?TB`~Dyp+u;a z8AR9#r*lKEzq>mxq?o~_%Z==p@jHwmj|!ceYVp5DDWrJNU2D7yqxr+tN{-pD>1ula z3z6EFGGb(xir<@q;2@iW95=S~BM*${{&YYjX#?LM01&9Ona{N3SAn&`;oqBhxEl$~ ziw^p`@0z~#d`ylTGT_dehN}=6jN~@IBOHQjgeXSA`a2m!=70-;a{+KJ0L}%#xd1r- zY(eb>!1ovcFhmq_@Kr}49C3VH+fn&=#dc$z$Nvkwb>!!)wwDfj-~ZsBwb^AmqgiNO#2? zQ~~Y3q%%KLoSvU4PTIdoaR$da3%DSgu=vtxWl?L5#JXdqvqCm{lSj<#jDx%uM0x`O`T7{N^+46Geq7`V@VWDc>p=v47 zn4kRX#BaT=P|Mzye2B*z2*fc5fbR0~8lw~*x``X3tbfTPq(M%R!vJXCEX@&k9_edd zn2l|h9^xIP2k~f~pq)z05NYyo3sxnTdIo;vZ)V~Q!Y-+HHjl%7E)OOU;WfcXQK`R_ z8x|Zaf9Hb@YC|1@6yg0)zeMPL6#mH*4=C*3H8qhe(eL-^GoMW%pXg6vr>5&%d^w1{ zFKiQtcDghe52zL#ep_mc5HX+13q`K!WEx=uOM-nLX&~fPW&7#!3g*}=4*DpE3tV46 z(j5R~0RSIM@BW=PY_!P4OOQ7WO<@RTM@i3sOYPh2)xR0$dr?_)(Z(zGZ^=>@ZM-hp zcwMydx@hC|KfaCE1)RBnGk-G*Uci|PIP+5*uZzl>|KV-CF5t{5oH^NzpR_GGg){A@ z5U>nH2I7ttX>OI5h_sH5nAw08l@fu@GZ@9xa1xDJ4(Nvs+CDV1+s`cYk-b2SQHcwp z6h|tY&c?lSyFu!TCq*(j%WDlm9-3!EmxiE`P-Kq~UFprnMYJTJlg1^$jxX_41{p|} zgSVJ*v93Y&G%WrM_1w4$l{t;r5--S@uVo|+0%G7e*@1v zJT0y{?gal?TqE&s;hBV|CPysH8=I|D?1iBqasU@n&LBRMGjLBDv7=!GX+e_3v;Zu> zJ3z_9w25P!2WWVY1((tG(2C@(T9as0f@Sq%W6^>batxj{mbFUEa=W^h4%-U^ zqLG^i2ED*Bo;s{^DUna3Eo6Czj>8KBfN&n$&?mz_Ht98h*RpL=@R8>sPiA{7%^ftk zDV_`~g@*ol`8=WM`l|+Tc(3?xS0CHM(b)iis*0wRTYL{ILy`SDe5AJAzw=)}my4>I zi>jH6s+o(bnSZ*A=>>H8p9)=?PoPVAt0%GIABowPLabP$7UYHNZx{kE+3H? z#gTX=cekJrCv6Uba7 z=Wf*+Ei=dON1&G$eAFGm+6>f|&=|J`K9Q>Q;BB7ngG%ww0BEzed6m%VB z{dTr0yfMAa_awLB-N;7sSNRjJ%nYO5ca04Im~>;Ju-8<&g1Lx^3GpsnjeNYq$!yZ& zSca2wyw3?)<7!5QX3f^{Cng;t;783)_+D4$U)|061EA1zf(%hQs`m-&+avzqC!sdd zs^+xi&EW8N7;~T#07MRxHbAQe8(;fDxUA81?nJn*-zBafPvx^)1|17-pmGBMMt&cM zJhzc#1;jsyByK*e+f#Fv7sW0YJmqgHNf$ijf~Q>Ylnb76!BhVA*P>i>D!b@Z_OHJj z?4ncIf4A47Ty!e?-}YLR3!ZYyQ%*WUobVJa;;@;w1jdvH+#dX$Y%%hiT z8SS*)`}-g9BCg)ZQWj)$c<54N(2nwHOS1j#yP#Ld?j0w$@VE==l|7v`9B&MYJ|#1< z4n0l&y=3Fbf8EU^_*9l?Y5j?tVE;8Y!9fAc4K7(Oq-ayZV9Z$gjR?`&X~fZyZ4<1^ zwr;mi6@>L26jmxK3iag2KA3(3CI;B2z|lu}Tk-QPx(ejduYsY?KKq9zyaunqQaRxT zMk&a*MZpB!i$a8ue+Atq_J*jLHZ~2vH-9lT$EEo!jawBUfOX5a?LluiNef=nCIlh6 z3M(cd#h@yiwjwrYNFzwc{kfXPFBI1{nIG%Q8p!^=T;0rIua&9?DwH-QVC@NxRtml~ zNJ7z;H4nC`MQ=tn8LT|S9gskXK$x#TzSbsfVH%5ii#i&JGQ<^ROyfTHv&tep`aLkp zcHyHFCyWXP09*$G=FCZ#eY)5>-nMe`hG|mAFCObdkIVLhl;dqzWm5CBy}lXNwVTQ?|S!os`gV)y=$*m zUVjn;Cqq&3Q<&Q^h>QO7AzzQGy5`K)8077Do<4p0)IDUxooCOUJ$2?7_$S_f*XcV? zoj!Z^ABVpk{^O$GF8U9wmjI;_AWs5pl>_tTKzBJ%Ta2A6!j^Hc<7{kU0d|0i&0%0W z>DV0_cDD>z$-&l8u(MgP^Z*>&2Mf|*e+oQIf*m_xiwI~+fJYydSl$Ir7{Q_raIhUL;{n||a90bqw1BJ4V67UQ zYXr+0!0~#pPzjE6ft*^fvj*I$2AiwE6)9L#0nV0#rDAYQ1Qr$n{Q~f?1nl5}TU@ZI z2wW-ztJ&Z*3luQHwnAV~{o}s$$EKnm=MH};j{ndX{9(uSLtXC=Q!PJ?Br)B*A2z5z zRBwGhzVLmX=lgc+_ba;Z%WJ#Q`Tgm=*Eb~q7`ZsN}-z*GZ%1qx3slPdpe$!m^&HUll zCGoHOgJ18uzOL{6T42Bw%D(R5eZ5Kjx@POEiG{E7JzsTLzgpFORZ;uZfD)5K|Ee|l z<(%ltV&<27BFxUp7j^zGrW{`sbbis@@Wn>?7uD=9#xtMi zt$*IY#H@^bUT*q)ME&`p^z)XY&le6a2;vt8f*1B&7iRa+vn>~jWEXmQ7q+MuYPUX{ zT=iwTgJ3cAw{G_Mhlg;u^YS^DlWIo<kmB@$X zlOK)@e0bRYVTi9Ul~Re$d9wHMQ?f2;R-7zuS>~XEpLp#pFAq1MlP*-Wi)kEsEYLWxg}G_xAqE+fDwr z=NxYrcfQ@%@b*sm+hspaW!@@Sf2(`;t&Nels!eZ=tKZ72e5<|at(C(!+f1mD;G2i8 zH(Pq&TxfYyAbWFw_vRk;&4#TvW*6RAQlomUZ*1w_sI7ftQt$?o{zhl=_4UZ>m6NZJ z4ZL1fiE2~6z9f3RjQRS|-fIUduQmH$n|Hic()n6{!)rrDsCxEm)0tNb*I(_KeRXr> z)f&^Q6Y5vdiCHCSrLC_H26x+*ek8QuPnB_QYw38koU?y^_8ZrbIoyNvFBW$ z_1uo`TwU$CDZ#k{`nm4p%NvoGt0!Nc4<%|8j$o0b)D&sGV1z*S=d!eoO zg{77k%49DL@m@HfzA$ute&P8N&-4A(=XZ6_*VjHjEqK0={(MjJxy{IP^)lqdz;pSA z=Q@A~>^h(Rq<+7(ocu)6iAzHScT3C2W;CX7m`qZB8 zsfOC8W(7|b(Vyx~KDoJosGWRra^Oj(;mJ*1nUnC%s#X?@=ygGF{^&4 zsPduSqKCE)AFPc(I2nA9Ird;@?}O_t4~{0`W4s4*b02KmdSGecfilknL)HflbPqJw zJ}@tMAZH2QpFF!8Ia@z@c6#7!q2X+g^6aMQYz_15#NPd_lkg7z{i}}qD?0BVZMZ*2 za(^rP{>99FrR(<%&fd2;0BbX@xbnWfqWg9Z@2!j9I~BaQVC>%R-g`?8@M_t; z5x@TqKo^sDUBi4Hkb@#N?-ZQ9#3+VR@B=6pf+}$vF_w2ykMTWb3m3ME6 z?yhCty)S|@qjz=s?^<`V!{tb&NQ3O%&X6oRG#TC zIENA(V|O+`y4iAPjqJ_|-ktfmcXn)@UR^j{;W<5OJ)P5bx;P436r3)l zpB_x!u^+jkY4VP_fjf!~cl0Um*b&{)=Z8)0ohpc)>h_=7aGa{{JT=~MDo=8%oqcL0 zbNh}1HZptr;mGYRrrQ_Pw+kw7A1Jzg@9<UYl-33e&wwN5?DR+*6F=l z3Zu95_;1;C+)~qd%S6L1`I1{Y*tc}EVHN8qMrTjtI8U^iPAsZVlvbV?EIP4&cym+y z=8X)rczCBmmsMUrRCN8p;qi78bUt{zWbAl<@A2K1$C6j%E?vTWa6_uBlT$@9=yfuDwF?h6e?C4 zURKa{S$FMa8-mNK>6eWsFU?!Jv_S@08Mw6EaOsHh(nHauEzC<7_AU`bFB$M(vgf#D zmIs+_xTHvONiX}7t<1%>>laVXUd(h}+-bUaU43y;E@Z6e;#|tbZSf;Z!6Rj3M}~Tj z9JCy1mK~Yr9qHWy`nSlt3*>qadD==YY$Ny7k~an98ajC*N!nTfIwnc01EdNAX;ewd zDI>KqNsD{L(kOAzPptI-O`XKK24b;<*vBUBWC(TZgsE9Vfs@c}B1~F=>Po_R5h0I4 zXpiGpg81?={75hUum#^D!!z3eK`wq^3%9p`Yw+M^t+=8#TyHIIOMt7TjL1w0XBDn^9^8$1ng&nyBSRVI%awnQ|QE0(t%Ai zrlt}zQH05-U^?RH)gZcJ3_aS5&e5XBeqZFFOLNhKTd4g7RFelaXGImaq55i3I|5Yh zS5rx3!4k5261g#etTrIWmB_p@WIGeNvWIAUXT*;PEl3ha>G2Eb3qGyqaN#2ki(C~*HPiVmcG0u|jvvpJwzg>~6*i4dND zNYIy)GOIv({D>rev59?YPYd!+XiBXEZ*eR^1UCZ@fMJ9@Wv^`4L!o~*fFWIk)0 zd0-~>V3z-IRm&rmkw?QTkJ0H*&}I=1(^G5yr#ZXNXiA=QH$T5U^n!5tMFZ_+f7Lm~ z9#T5>s(I(NQ1Kfy^_vRETlUD?QR=(w%JF}qLoC~Ik z&x0LbP`zJ9TQD}xH_NJTSp(lSE`0Ai{2^Zc6VLFo&hr;<{1+!55ZC|K)&F~7{trs_ zZ!+=!vFiUN;{I30#=ofw{-KUyV|_UQH%JAvnOwjl$^()*Iv^U$2aM|sV1~&8vb6<( zi;lMh3W0E%1JFykfU>n1aE$VRwbc@U!w>)(xe#!hOM!%pAlxeh41#jN-y#7rBNc!& zDh15+Dj+1Q253DpKrvkd@Oy~S5(U84)B);YCE!}F2NJXffM4AR=)0Q$-;^3i?=%DY zX_85;1%i%NfV!ju`Xbatdr+yqr_Im({ z<`QGp3(SapKvq8mRE_%q%f>ViE|>-A$^f9W&H;{K5HJm2$|+j_G;LwPJ+=sJuSWnO zYZ)*oRsg>x3SiUW$ZO~B6B0;2M5fZe+TsArRaYi}1w2=)MK z-sO7dKH!UHfHXY|NNNrNQx63aoX&wzQ&fnYPlMPrc@TN^G1f{xq%n^PaaOY+@$Lc$ z&(DVFl7$d2uLzP-b0OlPVu-E#ia;(OLa7o!WL-jt)mI9MY>Oa_VlhP3R1O&)lt5yO z6_DkrE49)ph{sqBNqS`v(N+z_SX2v{X;47219gz9uo7aSs35H5RrHQVh;pI{;@DI} z)(SNcj;aNsv1uXhP%C6RtAhy3^^iv1HU4n}B(u>0k+M4>W@Q&7WbKB~=1maAK@Y@U z)(eTYSs+gJwd$A^;$qn#3B>?}-)D#D=LR9ZeFr36It-EMMj$4a6B2h^M`eyd
y zt#=%Ava}44c+JGdN;t&xn0Wnr>LA;eWW_>%5 zs+}anQk;T>oAw}dM;fAx>_Z&X1IU^*3*mGgLQTm ztrSBg{pC>8yaXCNsDM(%l~B393Tks#LziPRD62pY4RCLD_9>w8xjHB>qlD^2DyUc2 z08P0Xq2l!>Xcw~?8mQGkDNVP@0$Qjw-3pBe+n|hAJybQ?4jo=KKx6qvC|BMI)tbAY zvcZ$dlnE*ln4!j&Ug*q-1)5#ygI3Y|p%$498a53;>C<+ob@6sb$q;m{c^JwW9)W6> zolrN;1>LS1g9^LdP{Y(X)W73_Msn_uswbglhYuQxOhIY6(@;g_4AkB^3yn?&pzQ5A zsG2(u)vdr1_5~<^F$~q`M4-NkC1|>187lFvKuw7#G{{+lQXAHxD&w8Dg$?NPVI0bm zB%qCkEvVD84UNZlpuECesIEQ*_4e;UhrMUS*?nl2_y8KvXQ7nwL#S+n0<#uSVUfCA z7{f|~se*a1SmLgjhymlaF=5&<7R<9=0826pVIl*hI@t1DwrW!4fE4$U>TVlChe((nWq)7 z&~6=!#;=DdG%A>9;J)Z`BaEG=hN-KYVXkfsEHTvr<0rK+Jx>SosoP*_haQ$(xZjl9 z0Si_dVbrcpnB3O|vu$_7mbpDJR+AalIM@qwE?QtB$^$%UKTKz|!Mxr9SSn$MiHjVt zu7)94U|<+V36H>Jlu?*b@?gX;24hUPVXDn>*l^(lELQJ@acz?@ZO8}nWT#+B@ifev zhcJ%M!e%xCuxvpPR;8SWS*#&gIJf|#AB17bvIxx4wgk(19_GYWU>epc%&l01ZClo1 z!nqjCu)hKGmu|u`tqGXawFRr%d?dt3!f3U-Fhy?)W}n@IMfcJ$wlD)zYY$*9XBL)- z9>OezkLhcua9>X@JUvZ=OHz4o6F(mw)G*-G5hh%|!h+lK*zj=u<17;g-smraJCj^^ zoX3OnnoHoiAwJx@B!H)KOX1=w5u9#&BH$ClDLdtGS#bs2+9ZWX1}ou=NEKW~mBELl zHSm~G4p)Yr)NU!@o+2eY*-#G`*;R03xB)&xX@X}ZYIs#gGu$$vfjhEK(K)SfrAh~P z*xKN0Aw8UP*bdi}cfj54M)>x4CtMisg0G36@hf}a8LJsCo$rO44=nIdnH5gc_rn!# z8{8fnfJa$_a8CQP>b@bkYi<~x*dKxOMWb-N&IR|m#^C8SH(bK>z)iIi@KiOD8kmI3 z(>}OO=!Y+Br{S#88F=IBEZmtt2aneT;XLy^+`aLS zQMgRD3b&fp;1T~ie7oR9RY@E^+`I{o4JF{*}gUASl}1vl>O!G+3~ zvP~IymE!1;SZiuEbtNbLjfXMUW#BFLg!LP4C==-IJ zOy-sJK@~zGmLW{~8br`7M^HCv5%PjMgso1ASnjJwu!1UtwCpvfs0k6*sS&)fW`u5C zgYYu7h*WJWLTu3?x&m#8Kw6J5x4kZF?Lb&Zjflu>UV3~E2Z@0dm?SZ_;n0|;~F93s>iM9_Tm2*q{?VdsVs(Z)pt zdoY4fFD@bMigywfQ3T((iqL!45Wd7ZBF))ANE+e@)4(PoxR5|lDcgvs~BB-^h;s*`O<7q1e$%ypjb|KT1-AMh~ z$0lD7GPrFmdK;X`)W9fG z9CjhQC~jmxGLEElc#yJ*31nLSS)|a1WT>W)Dw`iU9GXVPvNK3-c>t+xpF?`agUIB@ zJW|qo!Ke%)XRM3JY%qeXI#@zl%2tr!wkVSBUPUTnYe)wxhBVE7&gqLIHFKLt_kIGo zUAm1F>UNL@R}$%8+eKy=fBNy-G%~pNMQApIq@@p#3gIEruBD)&&Kwkbm5Ng5(@?IO zJXE5Gj-m>`(x(_GAD@LvYYI@35jM)SQiuxX6``myE=q1HM%ny4lwA8Yt3-fmY!;%N zL#3$rk_g44iBY<$a+J4Af=c-+P~sga%I5qg&{U10IAkbUqy}ZB)}kU(1&YyGhf+-{ zQNvsHs8~@0YB~C?*4~8jgw?1dMS~Jmw4jU~TGY%$D=NFGLsfC~D2u8c#mfJlzR-bE z9vV@O@-Ea`dpC;XF`+c^9+aEii`rINP{RH`R3M4*AM~R#WdkUw-i|W62T`Gz14Szs zMk(q>Q1(72Dmpica`ybF7P(O_-8d@Y@}T(Z6DU1%66LG)q0+rmC`rJNGNq?c@#&wa ztpSvLbPi=(4WgFwLnxMf0o7;@qntB~sCX)Z;t7^fywuO$ktiy)vWgPZ*HK-v7%E`e zKvAaSDB12N%3898iZpMdbo^gb%SqHQErp6z?V-5cX_R(qALZG}ppwN|lt_JuGCC+| zuja2=YA(8}l7_Z)=ApxrbToZCAFbpv(T+wIdd*&d<}9+&so`JU6-DUn4lY{gEk+v> zJhY#~M`u(5v~)m-HZPQ-Lx&=?c;z>Rp&V`ZNYK%E1)5!0iB{KFp9qkPm(5Z9>T3p(R?rQBq2S&Tm=5j#BFr%&VUUbB4K{ICi(2%32o_hahIv@eG zfF6htG^8}ydNJox-(|AP*N*A0q(B|l@UIITSKg4gvFm2~Ev{R|Cmk^L9jVinGwi#) z_f*{%y^_vp-HiYMYv8lR>;>wHf1Lg>W+GI`GYD(iL68`8Iy-`2jHdC&HN z674@P|0MRQ;R5xG;V*?>2ftB$m%x~QWc=*@i{!77Up2p{|FHg@^DkXk*}t#-L!Sds zasdYo;M0LXKA>a*2^K)d(>Z{v2oM(oVIH9711SMuDFxVNfL9Ed2rLP9rU9$QT^ zZa-EY#U}T#b|y|-hBK)ElL5!I<3f`--4c$zi}TU(Dj_~oi+8r+xke!2!CS+4;WmCb zmtZU*uxkhbEkR=;P~C*kAfVVJq$orOmnf+s#+r%d9wKj)7!DBiF`{k?@E4LAD@fT! zlBh+OOHtFj`&i5#dd`6 zJrY?wV%Ryt%DZ@mf3aGAF-3Q=yYFHZ1Ck6~Y)@P)re3mEe2J<05^l>SA@e0V*Cq7e zC7Du)>hO{b=TfKi(kk_(iSA3SBbN$iFI`@{)R?){SqBM}U#4lejM{mb$8nj$e_3kf zGRNL!66VpE=qUFZp5bVC;HZA`C}ZiUKY6r~etA}KxvTbaS=;3a8^q?hTok@Mx^;P1 z?lDfuv7qdjR(p(QIp!TZX7xhS@ngf3E2P{j;+0oeny=vZToG|zVF+Boid`X$K-7g- zQYx-=H(n|4x-vO*rG5HJ@#>Xp=_^gFtCn{lq57+I9aqr@uk!h>QY~MV*}ck{e|1&q z)d|Jb#yqIdcJ=ba)yBoE**jMUXxC`?*HGoxcv`Pf^j*UiKpmlLB#CQcIoFztujN%= z8`fN_H($$eUF)B_HXw&)v)8&f*U6;UZ8u$K>%LAja$R)hx~{eBIGO7-I;gh%IIZEh z*LYm%IG&z5KD=^Vx_3OzxZWbVo@#|g+OIbZT+f=kekO9gI(a>XeuG|J~$y#}nu=%EC(@jR_ zP3*u;f%ThG2`IJjW{>1%MdQt>&YK-WH%q2(jzw=ar*G!5PB^GAefczb5>cIVXXRV%kAQnyv^XcveW*S(~Z5Sv#!&+M`ZES+lQxZMR$rS z?~JPN>@wZSao!o6y;Hk>r~eS&ZHa840oQ94xNen&sd^o_~|nd=3R|Na8~_Y zGlsj=_PZ#)yWC56$#?Hc((kql?-ncW&Pw5?{=2yocZb4v>$dNv)9&$=+@q?wC!@W` zX}QOxh9~Clv2NZY%(-`&d#|zTUbg1mK+nCJ(R-!ewE_!H9`jAQe5ZCljXyhT?>_hbRhv?^2?1wYu4?7zkuIhX^;dt2Ue^?lO zczN$(Bl8iC1QAd^qA@%|wLjvSd_=MINGkb=gZ`*Q_-L&5(O?6D*Z*kP^Qb=jC}aCk zf9_+AC68rgkGZsu$t;g)I}x_|$3&ZtMJbPWaUbVYJsxa+T-)P?hdEZA`BhL>fpO?~Khznk@)V^Rb zAS2cn44xNQ;TLAMUQp-0NGW;IEqhU}eKFblBHxY_&%d}9f6+vFiOYQ{RQZyw`6YVK zOFrjIs=!N;Nu=}OrK-Z06Oxy$jV}wkUS1x0**N_&d-df&`enlsk}5vusXwRaIG3`Y zbNJ3lme0j@&zbXI;g!C^+C}R1uP|({_$OXzTznmifsHpqZ zuF$KT#H+!a*R;j2(W+nbYF<;CUrUd^rWT^48?VK)uUR;+^QEsxnqD__zs?$YeP-r$ z_1fzc1Ib9sEw<;az`|Rat+%MTZ+m!eD`an{THbc_zMb@-V!^k~@wa)D zcfv*Q=qul0G{56Fz0>G?Cp-I&J&ckaytB=I*Cu&a)c9_+^WCnYcRADV2BYt4)9;G6 zQQorml=bhW4et%x-;?^@i!Z%r*?o_n|9(XH{#q`YrGI~>|9$nu`;^7^-P`ZWX&)p@ zKCst(Aa4D@RDw3SKj4Nw2yK3#%lVLA{GqSvLzU*kOwWhT(GR&b=tS&8YxYAS=cDC{ zkBm(pvAaJC41c7V`G~sqQAmqc6r4|q&pTA-CC2ly!Sm*+^SqVw;naCO<71r#?XUZ| zvHjz$?PHhsV_D?m?VXQpd7p>`pG4)K(A{W`^^@TEC)$NiXj`9nsh=u&pQfun9d7wl z+WTqdUoF8;`SDL9ho2gXK4VpWHlzMbZTgJj{LDT3nQ9ZA%zS2NUl2iJ4G`c8zV@zuZSBFNS>FtczmclGiFbTs8T^Jn^-W~? z8$;?F7UNstC`MiPEv5Zix9wZG_uJ&+xAvWH#d+VY@xL?4zgrGqLVe%q#=oO4eCJDi zr=otJDgNGB{e4x-_X+d&#u!Ez{C;`kd*k8v?4ln6(jPSHAE>4uJR?6SW`A(90Y~Ns z3H!%b`H$v?A91P{(N8YrPcp+#+XFw@CVvtw{S;09)J6Z9 zBm7xY0czWRruF~q_57?1|D4|Xc{ukk(vrW#Wq+|~|3Ym7BV&It%>RY8`Ii~WFKX^D zl&W9c&A-Tden~oi@pOUW*e`1bznBXD%B}cosPV75uD{ZU|LU9mt7`SHieb>n`n5{@ zYoh*FYsas`!C#kszZ#c+Wv6}()@ese7TCfWHdM*H2&|D7lQ z-LVSl`+jGP|Lza{-kA73oAZaO_zzk2AKRKgZ00{C`(SkLkFJeBIEQ}(Ie*hi|3*{) z&D;Gq<;dUCGk=S*u+q%m;st-Vl>eRI@b`%E?*_-;S^mGzto&WQ_jis2aIaX=+j%y$ zP&R-znCxi3e-NEXI?&RRVYIn<1RWZ3{^`vBML`=w+pFB@XxBKJJ>@~GcP7xT;z@L( z$%p1UrqKF`|4;M&uW9KlTGAOnny=$ zo&z|T9L!p65r$*oVl;tbj62Q4Yzz4qVXFXR7!_jttEHHTnF>hdVvN~bjtR|3Ftoi2 zj6zU}vA0xVq9fH9c2tH@)8!b(Y%Y*6DKPx$I*fi-iSd=FFzMz7jAXbGV_I&)1Zm9} zYPANV+M@xsDJ^DsrxnBEwP707dW_T2j)_MM7+$UsqpR%1cssi=>NcEs+l1-jnlXXK zUJPZ>f{`uuVXQg*m`H^U!!QnDR9-tK){+mzI71k2!!SlWFoN+cI5Ek?QH)43hA|r4 zm>JJFCL8x)xK0LOsh`Az`+XRCXbPju`Y{gi3}#I~i{Xq1Fq(}yjJseSqm42Fp>+Xc z2!=8K%pxWuTEa-%mNDkB6-;P7ilMRAFbc&w#zQXvqJa$zdq0j*mnJZ-)-6n8bQ{B8 z+rj7=yBME5g-M(DFi9C3FzxMQf`S7KRhz}gorf4(lmaf(sUWK+7i{dIfzIhXP}EZh z@c0Z+r(uHLVHTKLE&#=Og%zN{bgwb{V**Z?Sr*+CP_0R|OAAhmB8l+TTTwtXiU zUgiTV-5A*Da)ZvbaWKx90C}}uP}e&NdS`uLYHtb@3#UPPo)8E)XFniL2J!C z80iUtjF|;cl?sEy{0JD+EP=}EQb4=10($aR!DRIsC^D^sMt=;PNp65y-X>VpoB%CD zTcD#`1khb{Z0<7dv zh&7dquo|8OpnAkudAuBJV^?68l~OFLzY^OxUxjrZRAb|1HCUcrj&-Xm0B=lzO|g_% zaa}#Ot51au%r#&snMSNkq{dox&De-bgH5axRLoZFaIFp->utkw1A44B-H!DLJFrQu z5i1()#2Qz-u)?KEAY0RetumXjmYH5`IAy`o1y-!Gr621Uv0>L%2Cy9ZAl8su4Y*B1 z*lqtXR=7KYHI$5E{mm|HW@rp6U3O#5G!HgZHG%b4{&`S4HHo$F_^{DpKbEbY#;P4N zSXX2go1o5N`ISMezH=U%>8t_LTMJkTcM)r9j9`QIB`kGu87t3;Vr>lM~0rRhwAeKmw~<*ur`bx3Q`6Bvx$L#ddj8*g$*_Yu>H}Wc3-Wwf_Jcna^Sw z*+Z;KoP!(IQ*kkOE{?lF!)XiXxDdAvNcJ&sq97Az%&>4XA~r6oE5ucead4LPB3zhR zjH4@fI9ihua0K|cwX^`oDJ{imT17bbXc=yMRg4odBshb-0_QhNaf-ouK)P3jGYe$6 zP)iMt=9J?U(OR6HUWbdylsI-zJx)EX!tq(8L`f5luTkUl!_7F~vIdu?X>pS3R-CC@ zhYL=%;iySHE}GK_*wh`kWrq>RTI$3#=62(pl_p%gvj@lXnQ^-9UYwWPhhs~d0P$cy zu4~bT3*^{w6zL#NW^~}J-XUBhF^pprIdQ6nQJmVS24V|iI4)%zrWAK>G9%3 zo0B+W;S_GB-jB=nPvczPX26o2#f8OlIC^^!ryQThIW|JLwSq8?qg=#ktPz|$xP(h2 zG=Q)yiZisW;{0Q4xXk)GPRiQAnH6zd$g+u}%_VS({Vg27NDD+;lQ_0(7pGoJ;arR~ zE+OB?@q05k{p;vcw0|CetDXKXQi0rY)FJpFSG%X+;TjlN`hB) zRp5txQhaQ?63;EJ#%r5oc+X%BKDj8z>lI|9v<^RGRN}MVdVJNE3U4WD#D^Q2@N~Nx zuM9Wi9h4TlNn!vv9j$oHgbwfCY{PFCw&R5=1Kwck!23f+d?wq8mzH=e+AEIiQ-iwtN7uSHGC{DhF`8X0a{ZW@9}TqlgR{LRI-gX zHt*nPhLZT~(k{M=wuiS=rSU9V4?v&F;FUWEct`Oeeyxc@;5c#!nh2HPrqT%8(maB& zlTK(1nF0S6gODj=5u}X;1hbt@2!#s?w45S>qJm4XcN7z%6Fh=5+Y6{U0)k5=BqVI5 z1pb1Epg$}l_{z%(>2?W0;;A5*;!=Xoa)er6O_2A?2)6ke!t#Nfz!ED6jrux*)2$@L zWAy}HK?8x;ZUwx3O@!2(njp?J6S_n#gn&*|JMASmg> z1cz*dux4@+IMbtqRDm6Em$(Vr&Eo{&u!mq+o*?*XlY~r_k09-yBABQAgwW13L9G0< zk*E$3?2b7?G!i7Rb3+7m)9uy;u(UtFiYsFKO_YD zDMZS_Fd)lPiB>U<7}4hu8RK-KY9pUGT)-s8>R3dswScG%vWc=XCy*2s5k+lWqH(O4 zIJ3?pW|@3ql|n$YScHG}J4=c5eGxGoxY*HJPFx$65IJiVL=B^o=$2Ozx6Rc=;jD~k z*sCG>1+~P;mxky4k{F_^h_sppqN1mfXrFE(Mt9XjHeW+jYg&km*cgylZYA>b z+KBpUJ<-?QPE1c3h>~On(ZuT{2Gw0e>QFaP#To}}xn|;WWiOG{Wg#~D`iRbLD>2Ts z5qV7mMBSjB=v^En4l6u>SUODXGL8@fUMG>VHA<8fjS;O4ZenC$oX7}!h$@Pg7_&?O zF@ulDotPqOH~mCU;S4cZKT8zZ0z_kIjyRJI60_nEkvr!FEaPEfcw>=BXD<Ogrie7%9#P>+ z6FsFfS@5YnDPv?B$U7!d#MGOC$N5d8BldPD-}=0aFc=6zpMk4SNkXE~ zq$F8YCCRup14Mk)B*u=6q$-w^hMQ_hu|Wli8>u5{sr4j}R7FY}8%Q&Z0ASo|BFz*v zld=sOQkA`hWC?3YVTz7KuV^DFJM<*SL^~-fp944?BT1v`B)M%}r0q~QNqA@?8OqHh ze|s+}Gj1VC<9(#6-XIWC+DNqi0g__gPO={ilA>ipB({E-q;`*xTrnpp!E%u-v-5zy z&rR~pjg!**9+E`lC7E=Sq@c@3qOMPoa{bZ@LmXD1p=hT^c=}4oF~P#ArfzN zfuvgvlf3y6QmST&BsMRT=)y1%NJU8$!5T@{vQDyL2q4HARCNm9uYq+!z*DdyiM zDYc7$wq%#&X-<)nLwh9Aa++kMWk@qs2c&FQmQ*!$NV4ou$PQ-&psRDqN(YVXh~$yi zsQF}0C4;Q#WRl&JEb{hN0a?f`B%4Yv^V^HanJ||u&Eb*F6(!_Q2cJyy3do9tkZk9O z$Wc`pnUlW)s29q~u0shqQ7$F(4V7fQr;6;0SCiB18nUEbPB!(|k~K9^fSRo%%fx>;2;ZA>p*s7 zm|PVdAzSF9&AL5aqHKZgAi?masCZ`YnV5kGYti^7rt;28Yl3qO5-`ZxlG rKY=+7KOH~yz<;dzz(2F>eJV1Q6-janc;sX=Ou8cL8_qSP9oK!94K)*7IcDgh$a z{hm|oQ%Y#lOWM@>of|(VoFq;1%{kvY^UVKqe*eSYtJXZb(Na3!dcV~mgnb(B+N~CY zt$eZ4v?V#&Iz!(*6}umvWR;$12X z7y3-reEsps+6&DVVH1kTY*(>lD3+Y~&t)o>%FM-*+fpofti_UdO0iTqwOI1milu6M zvE-joEY&+?@B0WPxr|@?2qn2a0*po;gggj&5V8r` zgls}KA)Am*$R=bHvI*IQY(h36n~+U-{kQ*##vr7Hi~eEBQdyW}ktYkIEQBm1Sr}wt z=1vB}gMlXko8foeiZ>=to+itaSB|K(p7p#R;||6$pRe^>PMt@Y^Y%76N&qGyW! z^{Rh-cG;>I{-NmKtz7Xm?%Z0Wu8K}s6)n?LQ9n}^RdZBPI$sstkEr7GW2$I)TopT? zP(}G-Ra{@9iuR{e(Nv_0+GkaiGI_{_hb`}hC&e=Yy-S3ak2Z~cfve5z4; zsBp+ACfJ)=%@#8R3|Zew%1nK0P#(~?##>vHGGG69o-#|{nw7cwR#v9#Tg$EOBy44i zWRsNbl77da)M~FZ*bIshTO+mx{Q=`|4;XJfU{vz-2TaPj$@(iyiba2gNm2C&@{~OW z?2v3`<$3)HX1qfClV;^XeZQ=%*0+{h+eyl5yTKy#|4ewD!LBh+r`*5S)c+hj_cw*+ zHnGLMSQlHGu7CZvgLIjDmS{>ZrI%=0&R(1~tVCniC7LOkL?c6@F(~Z|Gqq^ki=nOj zYiKin4{he(K%4m=q0RhH&}RN;Xfyu{w3+`E+RVR&HuJwhn|T`A%>NE;=HEk``9Gk| z{1dd970_n>PiQl%pw0Z>&}RMvw3+9j&HO*mX8vDjGyfm7nGy-!j05Oh12|p*oNEEs zI-qO=;NA#$UIo0HfXdeZ-xi>Hn^Y+Iw@X~Fzyz286JP>NfC(@GCcp%k025#WOn?b6 z0Vco%m_W`Gpj7kQ0);+{h$}X@c8zP-xOR>Oj{!tKXYR62hQFLd;t~o3B zS$q|etu`SkO=ph%SJ^ge?lzvLBpyp)Wf;)kb9|-RO8f$<^EzleQqVEB%bwI2hXbS@IkARL4kZ1t9!a&a_ zK(Y}?H37X5z;O(4egU|?1j>#B?vsG$6yQArRGtNV=Yi_40RPuO%{M^cJD~1+I5)u` z=-h~$uczP#U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WbZ+R}{5Hg=Z-x(76>(Kj z&Q}%jAml;FgOH1SxyYAI$R=bHvI*IQY(h36n~+V&CQRepOth8eoIk?(BfsnUBmD3j zrmeIt7{>V;YTHf%a$rpxVuB zU8BidySe`^+RfoyXgAuN8_AZ1c{aCcH-Us?Yp6FUO~DWTA=}zb=Y{kGZL0RH8G4{i zdHEpi#)P>(Rm~Y|H~3OeSS|v=E+EtmgnNL-pMXdbXub|aQ$XuYAl3`Cl^CGdIDn2d zK*9-hl>$93Ah{k$l>xowfMXNjd<}4I0m`-k?(Kl*b-?=uQ28d{dkd(32k`FzYIXsE z-9X)5Ah-_*y$gi*1C0j^g;L~w16MRK0Vco%m;e)C0!)AjFaajO1egF5U;<2l2`~XB zkn;qn-B7!sb|XZea@2P$aCIA3w{dkFSGRGUFPo4}$R=bHvI*IQY(h36n~+V&Cd_#g zrqOOV>ID6p3q3m{L#ESi@TH)z1cAs$KywI)HUO<*AodB+)(FI# zfQ|@|_#EhJ26|e6WE4o90D4;i#~HwR7I2*h%Dw{JUjv?R0PlA|<@bQ^2cY@_;J*aa zTm}MHfVv-n;8h@W4G7-=8h-{NzW~j@0?`+Z_6S4`}glxhz+KqwZVK^R!6IM83B`XtFXfMHh7$?V47PXsvs3fX5{g^5m z9#_TACsa|sSQXcosG|KTRWuc;qW0N|s?n$;{qC}+CH0Fp1MgrS%@76-9=b;f2Lo#| zux|c2E={3k&RFi@M(bwWSvLWLZq=9-ZK%ztS~s(-1|jUzEv57AQ!P=a(bj$f*3HfG zm!xcKHw|~0cGG=xyJCGv{}>8B#>3OCrYBdfeA+HAS+RV@%I8)n4ci83Hx`>!X+M>* zc7tyPg{2gTxq!CyK)ej-C$?p8~B%f!Jq2+c6;i z1<>&&w420nYBz#wI+y?xU;<2l2`~XBzyz286JP>NfC(@GCcp%kK+Y4Ob~8{n5Q1wu zxVnw2+qk-otJ~OwY(h36n~+V&CS((`3E6~fLN+0rkWHBLCQPH*W$ z7HCiYgK9SgFj~wu!>u7V<7?Q>zTvbRO~;Wi^rdg;%_L>D-Cz;=e7IZ z`k#a6TJ!8iNG{gfw43+|qpjO#Qku@+^k!YVslS+KovGV3`I_W2#nLI;Fp*0sS$A^w z$g~@)ZHm%RVa#;e4Zal=mR2Bf8t94vJ?DUA8<1)TdgFlOJHYup;Q9e5y8yT^0iMf% z_X<$?BjCFVR9^%9H-MU-fxs_7-LF9KMH3X8mw@ogK;tVwWG&FV4v205S~mi*SAn)o zK>RhJV+)Yj26Sx)dR_;TZNfC(@GCcp%k z025#WOn}-AwHr>m;j|l0yD^yto(EiF$t9LtV#y_zIbUMQgOCRy4?_OGWB$KmHX)mk zO~@u>6Q;F5F7yArFY~9p#KBQKi^tDdoJH-XfD>*;G~EZ@&7fQUm}oc|%(@wY zFO!K`vs80xrVMdgraemOX57uXsU1$b>62};E#36^Otz9MkZwYqfvih6-HG(lO~bMM zGl!IJN)8U1bt53%w68T;bU9|Qh_K+Pc_a2TjN0t7z=LPvq{XF%gIAo2y!{3Q@Q z4z!*GVyA$%GeG<-&~Y9}dm3xQ$K7CS((` z3E6~fLN+0rkWI)YWD~LpbKZn$tQ!*-%8#Fg^0MI1VdT%jc{e-}WO1)2LKdwXRdmX# zXql#p`kAVznxl%+`KsuCL=~qWQ$@q$Dyehw@7O_jrv;;`f7zj=)-MdltWZknc!o4RGH8JU;{8Ux3PA0pE*xP;6cT{4WDFuKSZvlySfUX@t&n_Uj8%XU1diMd20N{KNaMb~2^?*AF zcs|N2l)RxlUPqJ(FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^xoF_o-hT09a8xFhS zup17$VH2_m*@SFDHX)mkO~@u>6S4`}gls}K;W%r;G}?{ej1tZ$p?XEvhpx{AbA7U? z-OPhdqKc*>Rn$JKimj_vQSuk4IBO)hIO~D>%|KNJKs~1rYSKU*V^KOQWS}RKEfYu0 zNWB?56=k*l#nn5En1SDYls0 z;#rq&?l$bEBpyZ5@FGoa=e z5cmS9`w|Eq2SO)-@F}423=laBG@l2eUjeOO1F>&_w(o%W_dv%FK;i<>bqVOX3?#1r zsULyftAOJ=;7kFon?PAF;4U%a^Em+T8lch%_)39lm$^{#uQ%s>IRQTa6JP>NfC(@G zCcp%k025#WOn?b60Vco%m;e)C0+epJtcc5sxU6WLEi2+BrFlu|oL^F!2O$qa9)z6m zzzGj*LN+0rkWI)YWD~LpbKZn$q#N@DTWU^&h6W7{8X7dLwJb_EGH2XmJl`iDQqmB{ zN5QB$I7@{wgLyZDhr*1K_R{}nb^j$NZ%XOLHpHZx97#964Q5;ISx7fu?@nc1y6L=- ze$q|Vo;5>Hy6N_rtkbOqA?){AO6S`xm?mUWN<30p6DMDP;$-cKW{a>1MZjR$pcyP1 z0spH&%_bo58c??d2yO#H+kx=wK;s)g0%1M$5;$37tOF3`0f z=s5r+-v?430KFdqju7B%09;|9>=VG<2zZ(RZv?3P9Pl**)h&QO3e=nc0NfC(@GCcp%k025#WOd#h8(7K^@L+ggt4Xqnq)|^eq zCS((`3E6~fLN+0rkWI)YWD~Lp*@WY)3Da0NMk)$a6sRc3f+vE+6Ja8q2wAjl7I3^x z#-n`}i0(&|7XJE z2D`>QopS$PQ~z`DTx*`)2tma^8*04)fn!;P$2I~y{p{aQ2;Yn8M z3A?mNDPKHj??St@RNFgGf9@RZxwEa4AD#Qcs<{h`o>^5?WS5t$SiWNAb1Rg)xi<~T zqeHtlEfs^@8xw+Pk7e%O;A_ESIS&NC0zzK{;ctM(?|{hnK=Thk^a9X&35Z<=+O7cc zAAydmK;jzEbpz=68A$#Dq<#f@UzFk5tO1-(z*P#Axd8Wiz*7cz%YjNa;M)vTdjS7d zpvDUXDuB95AXo*2d_eeZps`vmlp;IjoUbb22VeqBfC(@GCcp%k025#WOn?b60Vco% zm;e)C0!)DJ4ObU&brDw=jkDE7ys$JcES>WUOYYi7~C$%jv(iqns&qTz8>?0iBM<%?BueTgdC zpHf9rkt%ARRq5yS`8l#^Suge5Hv{irCe5gtI0Zt=K)p$$+bq@881pFImT7;P-p#o4 zZUP3~r!nh(O*-$U&$r38cN5+v+gh%{yLsy+f7ZR5hP%wX>AtyLu|A}GJBsG*Ot+ez zT)Fb;zY06$<%7H%v(2J3ykW?A<_*3U9G2ZcWG~RX4~V`CwC)FD2Y|Nsf%peN$A>`T zW1#CG&~pe#9tKiJfZk65M+9(w4!D|uvKGJ{1w1DJZ!1uF8t}z{>T`g<4X9}c0&$@3 zTOim0ggSw60%*JlM7n_HZXntNwEhI|CYGdk!tO?V2 zH+h`)Aq!6XFjKqWiNF(KB1wo@^ll0@l|&Wwn}Mndpi@>w%QWnni5+uPQ956xnxkt; z^PnXa3q+J&z3JC*mSx+)S(K4mA&b*uGHQCmF z#G*8rAn9$~F|1QnK#k?2&ml7MoS6`Zja#244#f%MBp* zGtl-65dRhEc+mpK<|Ux(WuWI3Ah{Mutpj>D0FH9N=>}Yzfie%^-U@iUfVTputOR^j zK(!C>zYWw>1A(1DogWD90YWuExE5#(0Fn2A<~kr+53~k>*hfHH2#7ZT9brqMl=#HL zONlZ8Ccp%k025#WOn?b60Vco%m;e)C0!)AjFaaiz^91PKaAgr!7I9_KI9pl7%S!XI z(mB7ZG!H@^gggj2<$+Tk*o16CHX)mkO~@u>6Xv`L(|9*jPN@E6gKqg_;^Aa4?Pg>~&1%h=vEIu~)m~2PX53jfwZmCAqciO$e#Bzy z-ZV*R`Yv`R>()(oBE5CfaBTm~A+4K+ZG)!WSZ!03l3m7(r`_Od!C?6mNE`*aJ_CA= z0m(0b)R#c-alp|EI8Otv7*KW&aJK=TcEB44D!&DM9YA#_;7$%4`ar3(Y43!VtF;E6DCPJ}F4Hym%1*=U~v(f-sR z-Dc1~myM2nlP?Sz7&rH>&rB&6Be<>Bo=oFr+!;4}a%0@|Y?@>%iNm;QY`l_n(ZIX`JB5Xntuva!|_R4EOY75Z24RBNd z&Pu>l1(f*!_uGJ{8u0D}D*b?O4^UkL_-lch01$W&sH+2l^*|^Hgg*irLqMbfXbuC> zPk`1&Al3x5MS%F{Ku0r>XaTySK+g#v*$Sji1HCc8aUO7f1>?r`HH{m=NfC(@GCcp%k025#WOd#h8(7556BCaXonxb*Griho5=H;YwemQ9#gggj& z5OTT$r#r9-*@SFDHX)mkO~@w9c@w5FZn(yN;;gZ!F(C{7nsn%lJP~jrWYM^p2a`k< zO+~7xeO47)SF57rFR*adNUCUA57ci4sw#j^8M~%o&rIx?ql(h`s_1@16{jClMZ@Fj zA7EZiA-?-S&t#G2oD3d&GjLC~Bpfv(>t+PP%&>M%zGl>jVQ=%a=VVj7fnQWCOvB#h zY0nw{<7M|bg+f@(aBPGk>sv`F(6DqY>{k|a#+&u7?fJ=l?IzZF=A`P)}TLN{Otkbtp|)sp8kMIIX78*^Hl}>08D@hFaajO1egF5U;<2l2`~XBzyz286JP>NfC6S4`}gls}KA)Am*$R=bHvI*IQr1p&cp}Jx zC&I)z5whss$bVp$W~3fbKHQ`sRFi^HQ*o9Gk-@~9(Q;9i-9@ljnwcjvtVlDWClvlZ zt(!jE#&XNP$+2~_`j&Omx6PJy{;rfXX_+R}WMN0slupO$Z1y0Ciy?_z4hd1j0=~V+4qN4m39d z(H5XJ3dBwTZLL83G|&+P66b)fHlU{+NXCKGw?JUI1K|fU?Vg`wHOs5%69G zDz5>)8$k8Xux|Xn(7NHo8z#U6m;e)C0!)AjFaajO1egF5U;<2l2`~XBkoyE^-EdtI z*A;PH(KuUI#0yLF!qPdvurv=s9)vsyIp=|M9@vCzLN+0rkWI)YWE1AR3Da0NTxUOV z*4fjVkOhBDdUSa_5pW`8(Yjf{DK{BU_*sAnKm9{gFnVbS0|yT+%c_Kfx644h$>#8x zA->&$0lp24#t~^VS$Q(uo6INP{8=A~W5$deZHSIpc~Ez5rM(~vwmw0qNXz?9yz$#~=qz86H6lF1Nl96-$)Am9Y*N`asY2(1UgWk6#& z5OD*|n}MhYXx$3Lyg*w85U&I}s(^$K=z1IIsRoiefs`NU-2*uG0nT>;*M6Yv0N{Qf z@O%JxKLjd227CvB>O+A4Fi>*@2z(0E9i3b#1wWh2tBEoJCcp%k025#WOn?b60Vco% zm;e)C0!)AjFaaiz^8_g0P`;sjL-~gC4KHubCS((`3E6~fLN+0rkWI)YWD~Lp*@SGu zan^)sthxQbp~vDg~WBK}QxX>!p7EX5bwRr5Qq2xpx((Ko~P{Z^l}K(H)qayy<9-_Zj$` z4!mC|1e_JKO(?Rym6QT~Yfv80x5isrlQLibcb+m!-}G6+-UJMIQwgHY z^Vk}&9W36A&d{5{=aX#>-YH5`Y_T3@gn3byUrSvvVzW$dc<1g9G7J=VeKw|k^lUSO8U<(k60^t)t zV=E9j4K&Aq=sBRZ4T!Y^ZE+y}Ezr>cBszhv1kiI4NOl3KZlJdZa9jnP*8tZIpzLSB z{R`mv74W_|1(MB6fbV6X`W3*x7N}VV1U3M58-d`fKxh*Xehp~cGNn+8Y@3qvB?bHd zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l2~fP@(jqP`;?kmVwzP;Bm*&N#bAEAY z9)vsyc@T2e17|(33E6~fLN+0rkWI)Y%y|>05pTHEe&Q^(r#OMJ4>X8q5b;F7iI7F{ zrciT9R8hYfsHy-uWmU9H!=9PgF-H}p^HsVzx|cK$UQ)3zs=_!>$++|WvTRE@i!!rr zRBir@dCYFBwfCZR0~=?2SvUJ~W8H+kQ*14rux@sL-I;akrv74{b*8SwrT!dnRR2fS(K7JhRm%Sd@mR*6+omCXs!aHKA`n&AXW{u?F8a}pkoh^r~$fa zft~=6d=E&~0loEr<3qstG2l7~lpO-xhXKzK!22msc@*${22>vd{9gbyUjl*SK;20o zcnS!e0m5g2#`8erE1>ynAo>l^`W>vB*!Q$<1ebI$0Vco%m;e)C0!)AjFaajO1egF5 zU;<2l2{3`2CqV0l)(x#2S~s+AcxiJsA)Am*$R=bHvI*IQY(h36n~+V&CS((ivnEVq z-AJC;!?i69G}2*wF>B4p9JDd32k5smkOb~EUfKPDDV2E%R!3pe?i7GoLJ z+dS>>)4UmX=1uKz=8ZN7M}mOTH}__evf6I22>m}3_8aUP^K{Dndrke%!E>#7b|VBB z>nz=cnXb7oQ!UZXDYo{y`;?~j$4X4r8Tvz0@zBGQtkM&9X^~RCc+lR3c4?`$cb@*- zIofk)TPHs{_k~q+7ZyFUs;Fqd*qTIo^QPh0{+UCXH|6Dn=G|CqR;9gr1m+FC7c7

O}i`k9SFV-gx&zcZvu^P0g-or<{dzE7tp#Ji0uX1_5tyCfsXz6 z6-tQ%_vL&|0Y3l}U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WG;g@Jh--_uwrHHK zE#hURd0FY4Usjq2ArC?xgq-rgDGzKyHX)mkO~@u>6S4_&-h^q)8?LpVIBV@`PRN44 zCS5u+PXwF@Su}4r;3l)-J_VxvsX@xkpnonK9Rnv{NEui+LkTv^G(ETCV`HAAz>3 zK>QleaRW&F40Qbh^!y4WUz}PfrCyrK)eKC42`~XBzyz286JP>NfC(@GCcp%k025#W zOn?dGJOL^|-(>BN(7YLU=1q=<-Sn)PYAZPo^QQUj+N_&5-FF#wQxXp;Lz*|Wo)skz_9^vZUkJf0%e;3 z_iKP>3*g-bRBi`+uLIR@0RA_Dnzw+!J3!qIAh-(%?FPbofyRA63T&E*2xc6>2XT@khPKm*AUXR*V@M zhOBQTr9j^rln3;!@z&O)%-8>&r_9o~W@WCvm6iMTt>xBsk}};YVj5Li*Jv*_Dd#5ZuP`YV{S_uf)gQ=HhShFX;{}o}+XHVL z$EVuL-<+m2?JIA}x^~lfA^pIcsy%Cl9(dER&1ALfGR+iCrja4j7?kn{Z?|sDmc3@ zfif@Pt^hoh(+VYT)if?>U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WOd#h8(7K^@ zL+ggt4Xqnq+ni0vCS((`3E6~fLN+0rkWI)YWD~Lp*@WY)3Da0NMpt>D_s*dLJn3V^0;YLwr{Xp;l5PBa7e*iRo2t+;xnhyfeLqO|cAa(?3`xJ;D1v)+h632k9FMysef#h)@ zbrR@31vp}W^Bmx61IpR~cO3A13wS$#%1*$S0IDxSyzzHYyy3VTCcp%k025#WOn?b6 z0Vco%m;e)C0!)AjFaaiz`vfT7aA^^j7IA6OI9po8t4i~#(mB7XG!H@^gggj26Xv`L(}*`*YCmz7+Ebj61%FMdbP`VloCsMIZ}K6NsN(cvs%Usz z6+53$MfqY?TwkJ!_NP?QRHTa9XI08MeR7U0TGmTxRTK=Q87SS{dH)_YoB|;-@NP1a zWzv5BA$AsMU+u0_Z3?u9sNO)S8Gh!?SgJPxgRa<^mFMrG-e~)?Kl8?Sd77>EW1G@+ z<@;+{S8p2bGV`YU=61#UkUrB!!AzTnr&~==u3Y)FU0$+c`HGd#tx!r1nr_!@OmL7| z-jQzKIn>Y_d@pz`R{;NyK+RPka1E%t0R(>rLcajvUxCIKZSZVf0-9e2qOSn0Yk}B0 zplt&X-w1TP3M4iGU9SN>TY%&?AhjLneI0OA0Zt#_dK)OK2HZOVk00>v0V-<%UoB7_ z0Q~O(HFZFs9;gf23Z>vjwwzzilOKQyFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@G zx;J!h=-$x1p?kv%oU;kpgls}KA)Am*$R=bHvI*IQY(h36n{b>pVH)=)PZnIP&&B#? zS}QyeWWf_*;+zOsbZ-hZokSJ&n}Mndpi@>w%QWnni5+uPQ956xo};Tt^Pnab3!x4- z5@uPpJsb?7$w0ihs~$6iIWwY%mfx<~EE;p=29nS3bl^?jpqsH7c(ZRf>t=KY-UL3j z*&5d6D^1bk8}qC)bQ5N(X2MLeN>A9OMN0YNL3P7gDV9#zhPF~l)}5R^GVA6y!DhPt*Y4B)ng!yNS*cA( z)_i^EWNoL}B5XntFjzj;43>jH@DLC>41|vWjh_OMqd@a#K=c^U`UMdC5@31*E%IAUxCcp%k025#WOn?b60Vco%m;e)C0!)AjFaajO z1ah7LtsAZ@;<_TPD;j6(ig;mZURXNk7nbHh$b*mvA?G}B&I6l}O~@u>6S4`}glxi` zH(?s%C`LG*&FDNVKcJy6A+33;m?7_W+2i6G)IBx381wV+D+^=cn(6||hc!1`uK-3GgRsgX|psfms z`+$zO?S)dJ+RoJsOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l3FJHhDmPqI#6?A1 zR5Z>O74d@7yr6W>FDT7}kOv_TLe6*KdZ^AUn4Hwx@oJICj zCS<{1lMN%Bl?dEQiZc5@IWk~I&)-!0*jny_qDG3`hpLB!o z1%+i7kk}1$?FD-F0m*lP)PA7%0N|(xoI$|#5l|Ka+zo&y40t~QDjNY`6Hpxi{GS6g z%|M_9sEY!@6F{gH2%iQTV?g8_(A)+@+kw_N5c?Kr>j2`NKt}>dTm-thfSzt3*#qq+ z^%J!lPP$MfZ7eU8)`SyZm8Yxs^)A$ zHX)mkO~@u>6S4`}gls}KA)Am*$R-?TO_)ZzG0TGM^tn!7qOigfK^8m_CeDeFMeRoZ z18X!RRfzJTCG|Tucit%&H4A5{kTRHdb63x%K=Wg!jK-}%`?EA}et+i8^H#G(vRRbX z`gW4C+HSB2{XY}-8|)hMbjtmEP5sZobFF!Hqos7db(Ym2gnc^h+NWB2uG(!SpG;Sp zS{;|OZr*fWNI&qVYR{UX2i}yI4>E5A%$xR|(qQw(qSRIxtoiyEnyh^xvqjj1B4Dvx z)hw24KTVRZDkO~659|4Yofb-Dw zLdkV_dd@H9$q&E;m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28%^R*Q;@TpvEgENQ zi+EjWUROHj*Olf$$b*mvA;&y$%mbT{O~@u>6S4`}glxi`H(?s{hHLF7&RTn#6SCm1 zNtZ56Wasl|{!I&ofTUuEi6QG-Nh#2`2IT>LYrM5JDf9Jz z=P9%Fty!6?Z)N3veQUY3ouo{+%FtUR<*=mRF(|d#D-AY-V#L;ntwDdl_}c@HRLJ**ZgaSf;{Zd3cgldcrO(Qpy((+Ply$E!FnU)1Ny>d+u!OTSZstux^61d!P1Poo)vo{jDVk3s!>2JQE&t+n@x}rTsi8i-c!RG6 ziKP*6H34N2!2LPkX$HJ4KxGv0odBv^0sm>BCI$q~0d;LaupJ1+f$+CLV+RoF1ey~- z^divO1;o06wjLn<6VQm^b0tBcQBM@ zpmcNR{d?4K3WOm8@8+LlqrfOB7KVGcWkIDGk&jcT$vgj?Z+b7Ux_9qp29y^w?2bPF zW^}!qfI;_b%!)SI#sJ@DY`hz{dxov59p25>SY_6|n})kgyy?EVU9mo-do_yY?M%0t zo?N-|>Awm)4ci8JHwK$YseLwc?*?BB4$Ef1>j5ga0zNNLT>w7?~4rr?f;z6L}BOnn1x*C9dvVs;J)#R8;_-vMO4pVb4tL zn4^l)`6|^MT}zq=EeU>#s#z-O?VG;+%d+j@EXqi_xfgfl)@x<$tC=#tEzq7!?*=~2 z@bhlQ(!1F=oOh%7I1(hF)%tdlvf6I22>m}3_8aUP^K{Dndrke%!E>#7b|Wko>nvS( znXV0znQHO1&#=|j+^;mCUUEMq8wZeB19Uloo>Cy`0#fUN-ZH@PD&X7%xLyOwwgB#JfM+}4eI2NL1Ms~G zRKErI-vMfN0D)aV-R}DfrQqKCxu$^$FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x zoF_o>hT;vy8;Um+Z+LNYHX)mkO~@u>6S4`}gls}KA)Am*$R=bHj8{^b`yKIIqTX@cOt!Z({OD6%ptX#l7pt( z4I6FfPNfC(@GCcp%8p8&NR zE-d20A}%Z%XA6sXS!rHYI_H;_=0V7VkOv{BJaEbbn~+V&CS((`3E6~f!kjl@8tsM) z?I+GcdukK1;IB!M&dd`5Cqfpr8xFY1Y`9N>Xn*QX~5Q107{Rq8#XQ13jC8qz6cC1$w=J z<8{FK2H<)VD0>TVzXN!70N!0dY7pUF`_}>L;_5*NfC(@GCcp$} z-O#$Bbwlfh)(tOg&L(6NvI*IQY(h36n~+V&CS((`3E6~f!g1DwX{;L|3og^=GW|UI zDm)Q5^lBmvy~?6>GY>Y2Dw>K^QTwbawysu1$zNdNtdUgFvL2}43{+JBoicV!!=9Pg zF-H}p^HtIPh$>D$rizBg)jz<{oI-r}fx^ck&HpeAam$v7qh@5@jNROs)tdWaRfUzq zZpNW@lcQlb(Z~a~_N_COri!(_S=VmvHteP(9#V$XZmRYS8g^sVv>T@}bL|G-3ku8U zK&%;PYXRa>pyLFPXa%}X13fVyc@9Xm0ln>j;~T*F9pL&NDEk3$UjRIp0Pkg>@(SSl z5vaZj_^$yqH-Ny;K;17u@K+%8;!G$uF9D4&1CdvN=Cwd{9niV~h;0PgUIpTtfR5K@ z7D|aNGr6LH2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6ft)8m?S>1BxUh%|i^kc) zB3@IP*OborHKlnF@*w0v$Po`5@xUfz6S4`}gls}KA)7GgO_)Zzp>aavgzA-n#>qr7 zPWWr$*UX}JBmaRlnvp6*`OuOEYBz)K7mS*Pvs4%}7HyXp4bq1{YT8k$F--QascVetZq3ZSbJ z=&1sdJ|Oir&|3{Sb^*@afNL*MwhwT>3wZVe-UC48`+)BQp!!3=|1nT=5C|Ls>J9_J zBS7d=Abb>P{0xX31Dd}8qF(~7$AQ>MpzRb8KLd1}1rq0huCIWeuYu$@&~8%SQM(a2 zUsAyjzyz286JP>NfC(@GCcp%k025#WOn?b60Vco%sNGPzp>{*;~=hy;q_3juN z^w~B-k@c;l6zE&S_?B<}j1OtBnUy?jjo3bjtz?sx=k={cS*>p;DXZ-Ui_rfw;c% z^^ZA4`@edNF{*Yi-6-Y;JgC3 zegw*{0`6;o=LX>Y8L0dP@cjx@zo@{oc?qa_83?=r)U5@A>wwS(AiNQ1d=-dn0-9d~ zqFaF0Z9r^0(DphIe*@@v6G*%TbiD)g>;RIxfYffFcQ4?m1)Kq;P;$Mea9INrU;<2l z2`~XBzyz286JP>NfC(@GCcp%k025#WIZuG@4ObU&brDw=jkDE7ys|W}ES>W!OY(273)K~E~7w~ zd3d_j^yJEwPycmVPI>tq-i<*i|IcstZmc$|(&99x^=|OB;IO<8xIO^NJ_Ot!1D=C` z_YhEd81Nkdsy_w%M}eBpfWR@J?h7FJB@j9egiivEr+~;Ap!qBiJrA^g1;oAv+P(qe z-vJ%p1BoAit_wiVB_MejNL>MXe*_#!zNfC(@GCcp%k025#WOn?b60Vco%m;k*SdN=fL=-tq};q}eggls}KA)Am*$R=bH zvI*IQY(h36n~+U7&YCcdcaw+f?c12C6E6 zPFWQ#)39eIcFa*l>3o%Hj;D>(H-Rv9AyBVE%H}02a*{Z&TcT-h=KI`61{lz@%OkJ$MFOb*=biE7o><5wufYkdy?+1Y6BfuE~ zTn#{37;t|Acp3q36Hplee4hi=&G2sgE%a^#7j-ZJCcp%k025#WOn?b60Vco%m;e)C z0!)AjFoB#WK<|bti@36gD~rb2$|7D?nwOQ%`DLYf5b_}8LC7f&obtdXWD~Lp*@SFD zHX)lZ=S`T#yWvXviL=t4-h?dpYtp0>JP~jrWYN1R;EbCQ&G&(LGw7B-CLT@(({4s) z)U4K=8PmPoRPE)oZZO5hs62qdGG_Q3+RPfWEl-)HZ_UbFeJd+tIqjx)IO}F~rrr3y zoMo$hW46-t;@elVZryYz(pxtT$M(-0(zXFcF811ie_pBt#&4EQ}j%~l}b1?nnh7fQj( z**RZOzz@I#m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28wHslW3RHP-Eh85 zX0v??MEg^Ne49c4TsAuT4MxgjpxoTMKVvQyBe<>Bo=oK?Bjx6PUA?g=ecMUObbVTl zq#Ty?I|ijzlXVO>gJQ(ih^;|?!1&t(##;{#h-wXQ6n?Ud_AoLCp-T^f30wTMC=Dk34 zAJFx24OGVf|2d$h4G6RYb#Wm0EfDGe!ks{40?JL~B9$9Xx?uuLfC(@GCcp%k025#W zOn?b60Vco%m;e)C0=Z9s$_*D4aZwQ$6^*k+MZBOiFDRY!3rh1K88K|lNI%VvdhCMT}V~#3H=c}Uo5mlUiOcf1}tABurIfeM{#oBi1?vWs%^bNh4 zq^!0ZEJFX!g#8A)#yp*J|6Wu7bMRbip4|vf#yU$^W~Sq=eX1pVX|}E9!vdx0@Xv2& z-MqQmu$z*2NEzC^2@D!`W3gG4@}?1(H~3z#SS|yRD?sy)K=dlmdJTx(0NQ>A;=ceL zzXFLD3t-v21oXTNBwqniYk}T%fTIj>mIE#~P_`LxdjQW?!0QDnD*#_5P+bN1eL&6I zK%g3^+X)2yKxhvTt^pcrfk*&oeh-M&0j>1~g;Fe7!0U)I0Vco%m;e)C0!)AjFaajO z1egF5U;<2l2`~XBkn;p+-q5_Ec|-Gt<_#}w&L(6NvI*IQY(h36n~+V&CS((`3E6~f z!g1DwY0Mk5EGS-3ypSkf@I;UWPlSncB4p9Lk^jIZ%}6bxe3(i7+RdGJ3Pw%DSt>*Z z18;^GWrj6r(mr9?UG$ozWA5C*@+s8bRE+VhPzZfsjZkELD=7u~)}TC~Z;iLMCPf=t zW47ffv-GW5nX7N*TZ3-KX5h`R+D)HslWpxL`eA{seeHuv)7#&Fly&W<^FsQ8H&uJq z3_b9s%2$CV|Lx zpg9FZZvw5oK&<3Jd_D&dUjuYFA1ss-r4QzOK>NfC(@GCcp%k025#WOn?b6 z0Vco%m;e)C0@QA}u!swbxUguPEiB@7rFmWHoL^U(2O$qa9)uk8z%dVOLN+0rkWI)Y zWD~LpbKZn$v>Pt8pEwKcsZGd&za~YxJe~+R5wfV=EZ~rvj0gNIz<{5A;RBPG?lf-( z?pc;q3kPqPfq64Vvu74)?#qGn3V^ym29#yIx}zLYag_ApMrez^6?X6Cf_t9j}9&0v>X^D-%Npg z(|%$E@(sQhM3!|xVgt~%5$JgpNNxgBuK~SV0EZWFRsgO_psWgT`vA||fVUc`+zI&n zK=mHLUjx+C0)YTf_Z|?e148vcI0!U;1VlnWa{~|!1FfF`u|}Y+35Z94j?aNaGtkun z^hAN=3CK68R?0VmD?69~6JP>NfC(@GCcp%k025#WOn?b60Vco%m_W`GpnOC5hVl*N z8_GAlzB!waO~@u>6S4`}gls}KA)Am*$R=bHvI)mo6Q+@G46@*AeXiCQ)K+*R$bu)r z#5obNDBt8mC{e}f$5heqxGHu&p^Eaws<^&H741){qNzv~wa=;)bovAxS+uN|`t_TE zcQBP^pm=lV{d-h#3WPBO_vV%l(=W!1_MJkCH{){ZO~9ZFHfCM0G1%@DZ$@Y8P0y(Z zZ6(!zQkptK7qc$jG~8wCP4~_1iuEC@K?u8IDV_iDbgSvfl`EgN%S%=)U$OGJ6-u|y zWVP#$Pr>7M*&;OY#y)uH%@n2OOQSVkf8u2AiDo=eC;|@4Da~Oy1EkIZz2^Z(JK&51 zu5W>|4#3?BcoKm3B2d`{__~4W9>D(-P?H1#*MYhe5WES5dVz4spWxUyfXEu4*$G5T zfmRm~TMx9A0r7I6!wn=h16>}VXDg8O0;vk1w-Ru?2{_;SQ=#N~=TA9bQNRzt1egF5 zU;<2l2`~XBzyz286JP>NfC(@GCcp%k0KFToEaJ){t}Gg7D~ot-X+6>%eI5FC^P9s)hrn3 zKVx)Tt-X;|9Wei9v^5|6{ncap9s9y86apTL85V}DZzZKb-x@NOZf5AuwPqiV<5Qwg&wH<8KcbZ#`hd?3!Vw-R#SabmOT0ldb$GNH?GQ z*G#g`(EXLEng}z=Dm`JB7AfV62kl*GmzHXK=jqR#qdj-Fb@HQgUsyGFVbL?Iii-Z~ z&eUJbv(D6?mai!>Q!Jgb4Q-{AtUEb-q|%K+Y5BX`rJLU-;G~mo@U;znXpllD| zt^qu?fHweCz6bc~fa-d{9|USX0s=NL<40x_Uy7B%<>4w8@m;e)C z0!)AjFaajO1egF5U;<2l2`~XBzyz2;?h~MNL+OUn4W%1OH@vVpn~+V&CS((`3E6~f zLN+0rkWI)YWD~Lp$5|7mk!~beaNdoHtMsX>@I;UWPXyzHIT5lb-4t-Z&4`Bkz`7Z9 z%O4X7CxcNpgJqli;pCq?NAiqx&pcrm1Jh>ETj(Qp27lSnc8z{t;!W*v;?47LdnB7h z>6>~pNm*?-ScLwc3HuFpjd?od{=KID=is^4Ji8J8i*=T+!c5mxn5h=`Pk*vig&$Iy zc6utZF5Yw}(u+3@$M(-0QoJcIH{CAT7;Gk`)+yb-b13r$-wPH?67XIJDpP>(CQ#iA z_)8vwW#a$>Yk)c@5G(~kE+D)fXed_d3J zK(ZQ0?F4%LfMYM<+y}Vc1M}fu@K%^CDJ`F@;K6S4`}gls}KA)Am*$R=bHvI)mo6Q)sageVmp;q7d2_d6Hzo0qGNgG^>lrlc z#*8UH-5U*=54*wlg2mznf}4Sm2MBKk8ofZI0%)!TqE$ev4~V@Dv{eJ~oj`{lNbCW+ zYJi?vAQ=Es?*YAafa3$e`61x?7$`dkxDNrI!+`e)Q28m~I|@{P2KbKwHD3UMFM+z_ zK=33GIt7H!0F7s1-bBvRyb)a1!33B96JP>NfC(@GCcp%k025#WOn?b60Vco%a-IOq z8?G(l+9Iwk8fR;Zcui?uQ#$9@l;%OmgOCRyM?7%E1DlXd$R=bHvI*IQY{Hy3VH)#> zYwah_T6>xkvf!^tm(IWw0VhHh%^Ud-Y|@O>BFcxE)UVy#d8c61M4Y94@b8Ibp<|<8}o%tr~+D+$$^aF3I_N*Ct;7!A} zLE4SQW>s2Fj6l1=_kzOG4n*QW^S3~>18D68VhNz_A`tHaI=X>G576}!(31p`*MU?D z=)DOzeg&K_&V^$05>WOs;C==0tOdO5fXWSkZzE9sD&XG))Vu}+wg7e8fZ%o@^g0lJ z1895`h`a?fzXL>f0Ij>`7D}<*b926+fFFPfFaajO1egF5U;<2l2`~XBzyz286JP>N zfC(@GYB$twsNGPzp?1TIo3jbogls}KA)Am*$R=bHvI*IQY(h36n{b>pVH)kmC=0IB z=Q@28g%zF%vfzm@aZZFRYBviwrV^C_fR~l>v#fYsDTZ8_9@wW$zw;mXYdGlxePnzNCXn)eIJczAilad3$z7*_I6nnkM}e}>0QWJ#^9A7j5~w^5_)Y@VrvU#Mpyn(PI1ki) z1q8naLf-)4?|{bdfyfU)^93M!3240x#I68sKLYWqK*u$hH;EfGZ#eLV2`~XBzyz28 z6JP>NfC(@GCcp%k025#WOn?dGJ^`9HTwBDoMO<4n&ej(3y3)L^bk46U&4Z8!ArC^1 zdEl4_HX)mkO~@u>6S4`}ggI}*H0BN0+E1Ld_B1DC!C#Xuox~FXCqfp@n|#b7I^88)g8*RU2v)poUCMm1!28+=D zGhx5Mt}#!i+`re<{~SEmnrAm!O6OZ=Sq(zir{k`DswJM9YwK?M8>MN(yPK@m8Tvz0 z@zBGQtkM&9X^~RCc+lR3c4?`$cb@*-Iofk)TPHs{_k~q+7ZyFUs;I~=FIlmC#meVa zC=GX+deeP#yJCGvf4!o;e!A842M4(~0`5&i#US_Qx1DzVFHONfC(@GCcp%k025#WOn?b60Vco%a-IO? z8_GA7Zz$hTzTqX#*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL?|djeL_Q3$E7ZYJD@c z6`lyP;E6DCPJ}GVH-(x{qKf*>Kvf0MDXXGo8urY@jyb9*ov+f*(cPqZaFdFKP=y$% z-Q0P9S++%-MH!hlqdt9Rx#rp6yWhIuQ;osMn{;pb*U!-ZVX}K?`RzaPBB7ZqW}8rC zeJd#i`qpr3)Xm68-pm-qkvIE>({4s*b&pK0gV)8X7 zW{Ra#wxO+*l65C%53Ake>o1(Fz0eGqMkvCXX4h@XjW&F!d@=NuCQV<70IAP`-e$n@ zCEz>`xK09PrvUdEz;hPxo(C$w0(@Ts)!zXA?|_=`fxr(y-31_c2?$*V!dHOCAA!hK zp!pgQy#ciT48(o`+I|J%FFp*t<|QEUGSKx3(6bgut^-mVfZmOO!won$KU^reJP+r5 zF#$gS6JP>NfC(@GCcp%k025#WOn?b60Vco%m;e)C0#t6esECV-xTt8HEh^$=rFmKD zoL^R&2O$qa9)z6oz$p)GLN+0rkWI)YWD~LpbKZn$lp8LxpE!%`sZ7X%za}L*!4m-| zLKc;q0?xP@(R`mf-H!a*{ftpka59*7GwLSFpzmJzn|w29|ET^`e^bmB8q-_hJv%qk zbw|gd^lc|8qv_n#=Ek{kY$*-MQ&bq<3x_j_sd0q;u2lGg;HvG#N`b z_*M{DwgIl~K-uen`whVJCg6PwsC)NfC(@GCcp%k025#WOn?b60Vco%a-IOC8%j5nZYbSQ zy5Tj=*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL?|djdWv@1y|{FmA-+x3Qq)C@I;t6 zCqfpb8xFR~Y^YCxXn$&Gn`GJO$T#`In1OIJrVa{}7icI$r!g9j{!WFP8Dl8i?8%LA zpD9QHDIg(V~r3YV671&L<}`Ztx-b^SWA#vB2)|*Yt&c+)EYHn z#QJ@n19l;Vl1kciyU&^UIN>B|l4s6&`p^4uUa1Fx^g}@AVPN19AiD|3JqqNv07F{= zTMJ-s1sq<$xgBu%0QU~S;|E%H0p0-MYy0PVoj>@`6<pJY2g&lJg(KJsHgEuSU_^pbFEKvRd#^u!GyRQ+3 z;oD;M{z$`R$|d0_qqJ@ot3HfztZ#RyZ!yx>AB#u-POY2k$I-f}(3l(Vo`1Hq-TNuK=Na zK=@T4x*zC#4T!xCbRPubhk)M0K;lgxc?3wk1*DGwnYV#~cYy2(Aonhie-9XXAFzE0 z*wcWcA8=*>*J;2#0C)z0mMq}?0`TPk|5+fA2ZHB-_90j|p~ipVf(9nQ1egF5U;<2l z2`~XBzyz286JP>NfC(@GCcp$Lo&c>IS~s+AXx-4d;Wf?Kgls}KA)Am*$R=bHvI*IQ zY(h36n~+U7!J4p$byF<~F4N~SeI0!jo(PiQi7`a+0?3fO!ZjYYSlkW@y1kbF~~P)VvSLrr){g`+1j>Ro~vyoc|wUd zf5C^+SxoY3Z97F?ZPlBE_MZ{^^;VTt8l~aA#^LASx#lXX0mEa=v$Z)gGt@aU(@nv9 z|HTq{6XH$wkYlQOruNWuJap3(vu=@9w^(joQm}WSRkuvtTcVPDwIH#q1w!`$ z;roH;13>3`Aod{8{SXj;80dWjNNfUPJKzb{Xc?=kM9LPQaWVT!1prX4+DV+5R3xt2Y^r~5bgq^F`)B}f2r5Sx+&g> ziZ80*2VeqBfC(@GCcp%k025#WOn?b60Vco%m;e)C0u*n!w1`WKxU^`3EiK|zrFm89 zieFWl2O$qa9)uk7z#$K8LN+0rkWI)YWD~LpE8c`f#2YTPpFB(LDNaa&za~|>DxL^9 z5y~ju%;$)klE?eZ$9SJLLU;a#pW$-%=Hfle%j)6a?MiTOO0sM2P;D7gS#FEfzeDk6 z;)*wag;=Aaa3+O%u#NP&b-vJUQfaJSC>OCO+K9KnU82AXtehlP30rIDS zp)-JO5U^(f#}|Mz2e{4x?mXZ*2eb?U-o`peHa5V&4hYzRU=z^p074spuoH+j1D!4) zwh`!d1M$s3ucxkFm)KUvH4RLF2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bgJOPR~ z6mKZrP`sgd!>gOK3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g@kTERuGHsBeL-b~ zCxRq+B21nWp^V~94P+8U9KTf&kp+s_u}BfkOB8W-sUrGrQ$$yTBHHd$DCeA)b0pET z;evNFEa05KgPAlJOE;I_zeWwGR_IIcZhnYOQ?I%+=4-goyP0s_O;E4-HPx~@*k=5^ zoA_gOmcCQ)Zk~TKRCezsa)p66gXgx&<{xVVb7Xa3&J45hPb*jc`EQeQB3lZ)8?$Aa zJouEMeK`R9P4J%Ftpuy+8C5a4_n zaD@SP1n@+GmIHvd6YzBb{umH=0|<5l?L9y!4up>a(O#hQI1oz!-6w%~66oy%5-A}0 zA&^P~>3(=OnGC%f4!mIkOn?b60Vco%m;e)C0!)AjFaajO1egF5U;>p-fZh#P7I9?} zR~AjMl|{U+G_NaN@#{+SAml;FgOFn$IOc&($R=bHvI*IQY(h3+#hb8*cf*zTlV_zp zy$MP1*Q7~j* z48Hgixoguy@v?h29jB|zvoyV?26~NknyFv1ps?h|^~Y`;op&SP-9(P)%~tKNO;i7x zWD*v=Jb3-3`}Ek?yaW=j1YM@H7D}4#2ws@Hqj0GZ1hA!Hqz>8whO%!X6;H4d`qEVy!^87l>~MdVN4*2axmw zsa-%i0A$*Lfgq559>}%-q+XZr_({bt<;f4g1egF5U;<2l2`~XBzyz286JP>NfC(@G zCcp%k0L2@MHxzFu-cY>Z#m(7-Y(h36n~+V&CS((`3E6~fLN+0rkWDzjny`p?qmu+z z>T{*OfyxR`1WE8jm^>#!8O57g4!IfCfFGDQ1-F`U5phZwcQXoOrWChk+10F?!FkuJbb>jfc?IC^*YD>^SGda2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6 zfr=+U>xS!!xUPuniYC~)B3@ORSCy{#Ri$|l@*w0v$RQ6L^1vo!6S4`}gls}KA)Bz` zO<2Ub;X3=tv(BE@ge3TD(xa>9iGUNKjMfe3+mtrjr&jdcHnPW|`5?N2(Ki?W^M{SW zsS%*Dl?hmu<#Jy4iI#>1J$Z-PoGu zS(tL0xdwW6=?SYq3uA}2Sj%Ooqiy;3+N62@iw402qc~d zlI=jM14xH}%*(()7|2F|TolM30ERjN+d;s72yh$*oNofIBY^uYNH?Bilx{@D7gX>A zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@GN;i~lDBV!Hp>)H`nzISngls}KA)Am* z$R=bHvI*IQY(h36n{a|PVG-#@NP?^Mxk|r^x(ZJON$^CNJSRdKrJFj)B#P*2P(<6E zirBnb5smji#960PM9&7GVpJY2g&lJg(KJsHgEuSU_^pbFEKvRdM&{Jx zyRW%uzRTibF3QDrN4Y#4r6loY{B)Gns`_G7gz@4pJ3jYm-AvR`H?H1!me%b*mAg9P zTgqam(kM%B96rYok{6x#=drB_v0jsu=YZ4@kZ$}b6dM~b zunx%Dfm{=icK|~h0NX=={b9iI2;kfVxE=-ETL8~ipye^Z`#9ix;-~dG|C2xEiUuaY z1egF5U;<2l2`~XBzyz286JP>NfC(@GCcp$Lo&dEQE-d20A}%bNU<-?QNoihEy5g6V z=0V7VkOv_rJaEDTn~+V&CS((`3E6~f!iqOx5$%Qx?I+JddukJs;IB!MPR|noCqfyu z8|fcdqZzG2R0A#PV(q5je(ji9ILm~ggn2iRV8-=UPwhs&!CY;swiqsKjq*J0-&OK# zZCfqR)wYs+y|x{nVK;488+NlAA4X>}$>+CI4%m(X_O}7YJAm^9;CdHuzXy2U2UX953iAn+Uz z+yk`l1wtNfC(@GCcp%k z025#WOn~wYmltt)5tkQDu;oR(v@|a*UGYmx^C09w$b*oR9ysZNO~@u>6S4`}gls}K zVa1!Uh>~3D5ph4i-;!zPJ}YbH#HDS6mk4kMMM@TV#gvyG%r!a*`Y4T@;HQ=y=9UeJ+5&j#JFesl31Or{%PgP zKU<}xEACja@~#zfb8~@sW3rg#o-?BmZ}7b!u|$E`0ie4Rh<5?KF(B~NfC(@GCcp$Lo&d!giZ>K*DBe)K;T6u=gls}KA)Am*$R=bHvI*IQY(h36 zn~+U7!J4p$cvFQd<#VMzSL#z);fWv#9Z!Twb0U;cys1}R5=C@u1X{NN{gNVjuEU;L z*fB>DP4g7GIhvPL2QO)jKt#pen_&-UdATK=#U)ucSGQ~GR8PiS##@d0*C^ghIPqrp z)x?{znRpX-{@l`c65`Fv_kLY=@uuT+m3fxt*VMqTu}(AfOBNKC+_?VOjiVE9%$8|# zThLH!x=pctkXW3m#L^5TTtIRoka7d*%|ONj3~U3kEkLdn$a{gI?SSoRz`hf3JOenN z1zfuU_j7<}574p~@V)@}UIhFv0fAS5;69-JRUot<2)_nIUk5r50NP(_&Orqs!wTL%5Vkh>l`)BI27nc73s)fqHX%(_KZ-D0_UNx|NQR^2jnZ=Lqs zIqGw7G*7){?%k{AE^N4cRYSwy#?}mGifcEK!+U0psNF=i6wJF3&~763%jVq}<;edj zuHE2!L19S(nLc131!O-2a%mvn4-91h+Zn+ADd6}FaDEQBz69J~0iLgcmTv&>w}9_E zz<=*ApxCShg7*RK_XD8^fbe=C`XJEx5DNfC(@GCcp%k025#WOrYWkP`jaa zL+ysz4YeCy+ni0vCS((`3E6~fLN+0rkWI)YWD~Lp*@P3U35#eqMoDm;KG*5%DXj2B z;M}XpH14O2+6@QXls4R_R`lIg;M)}ZbNSflH#I_Ef^u{1{)|{7MsZuMKKYuI8%2G& zaa6a}>hrJZO@F}4>xF8uR9l2BXOPanXfeu|n^Ro5!S{l`vJ1!ufT1?Pwg<591spE`&KCjKOMv?oz_Sl%c@^;P z2Yjyq{?~!PK_GYtXg>^u-UPx&faqI5=P@AmHqiYJ5I+I*z6&JY1Cs9psSkkkM?mId zVBixVdkV;%0rH;$L!SY*F93TE%8lbJl^agFVFFBm2`~XBzyz286JP>NfC(@GCcp%k z025#Wl}~`m4Hp%0Q4tpvO|V5pyr48MC|&UjO7kG(LCAxU^Bp+fflbIJWD~Lp*@SFD zHetn^u!wR)+l00WwJQT{lgVV8@YlqzSw`ii4l0Qvx*8PGcBdjXuU16kJmuOY`ghDtEniAX0Yo=1RkE8dG6; zWb-CyEEH;ta^G#EFmLd^V6l7yIKBm(-vO?B{}qzoDskk1>6VzwO;4x{MU+K#*-g_2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?b60h%{7Z)o1oyrFr+3!AeE*@SFDHX)mkO~@u>6S4`}gls}KA)9c5HDM9+ zrdkpdFDPEYHkOWVJ$#Wu<(Y%phlR&MRqll(?iWt0E5yx*;L}Y;?b}Ujv^Abgz zU8>N|(afY8m`N9#HwE`=$4taoCPWG5%~*|rS$+k*X4xob)fCleM!S>R%~c29TvfYS zJto?X>-B%Nv~K#P+!bj%UUu!K|5WjTH?6zYjXdzCai6hJv8k3LUR~kN5hmW?i@{+z z2zU+wEr$W`n}F{K;C~AU90P)H1MTksp%Xy(T_E}%(D^NfC(@GCQ$JN=-qH-5my#*Wzhs% zS;PxV^TN^%csB+A)O3}y&BMXlm7v}jN2{^?u;r8 z6wvuJ5ZeiKKLf;{1$uV_iRXai9w4Nfc6dT8`?LtZ)o4}3g>J>HX)mkO~@u>6S4`}gls}KA)Am* z$R?a%O<2Uf(PLpO=Y4RwKD`y52$G<;A|}m=P)7TvhLdhepYkJ#o(=G6pxqSSFqfu4 zzA5ob}q8c{E%$u>9dlUTNmzKzLH_KhNrl#rUnP$BZ*6F6Ec{fcl>lRsci{<7e1$!4- zb<5Pfb=q_1sL#F8JoT2jcdweeu;KPq4GmUl>54m6th{T59J#{Wo56EiW%G}<*UReb zXPAwDTDkJie|zDe_cHUwB)9F=k7Di(z8EZ)kATp}K=>0NdJ5<~1H?WBx<3Qrp98&L z0*SAH6V7UKeY-nTr~j025#WOn?b60Vco%m;e)C z0!)AjFaajO1egF5sCWW2Z@9LIYm2zHXo9UR;^n1zdFhH@UYZ9X4?-S$Ywc-HVDQ7FF(>$I^4FBa4>u#rXx`MTGKnHOHUh2N zfPP64J=bB+EbN%0h^Bc8Zgy8jyNNz`v!&-dXg42x{&?B7n~u{}=2@CuQvd{ z`NP1_n}972*pC8^Uch-Aa3uiuNx+i?TKWKQ3h;dh_|rh39|&fE_R~OU00?6amoAzG zI==v7IiUM25YGd>=b+srhN#_e;0+UC0!)AjFaajO1egF5U;<2l2`~XBzyz286R3Ow z)NZKVP`jaaL+yqaH)j*F3E6~fLN+0rkWI)YWD~Lp*@SFDHsJ(o!Xny@?pjuMQ*q#w z6xt!QL)ak`aAsH;wVPVFB#LNWqKLCg714K_BDxwB(RQaIHm_Dh<2~?j)cyDH^j4?-eX2JTPQ7WnntG#7!qHhw^7+X(Q{>fFy-8^Q z8L?k)RavD`8s2Lheh!{%uCf|nz?f%iGRzEBhM8`Pe|NK`@9=!N>%e<2lwG|U%oJB| zB8T_P8d1GzZZ4R3W426_2g8O@OuWGtgT`|2d}uanf#iKa>V6>o0FYS^3_J*A9|Cd@ z1NldQp-q5oGhp`sj%|Rm1#qN zfC(@GCcp%k025#WRByPrh>MH3xM+ecF5-oyd12{_Us#$4ArC?xgq-uhIS*_?HX)mk zO~@u>6S4^_-h@Te8za>TsuNTvCZIaOk0c2L&&?lvZkAEK;fR~k#{1MlODb?~3jVo# zY!sXtp({bV8Og9&uKF?4%6nU;{y{o76VAEWbv5VaylYc#=O%S{zGZOhujH<$4!v1+ z=cegQap$Jz#Oc|ib8ZBjo5)MLLg%JhZa$@gtsHjsY@7&-yil7PJraHIg|hkz>$xcdQ52531Acn1LAAmGmefiHky4ro6M zgz`Z691tA>Ivallzs3f1uLI(CptlJ~IDq5^Ams$o%|ONl3~U6lZXmb$SM|EQ=T}_L zzyz286JP>NfC(@GCcp%k025#WOn?b60Vco%m_Wr7pmRg#hRzM08#*_ z6S4`}gls}KA)Am*$R=bHvI!?x6Bco91V`3zWDUJ5x*b2*?O=2hGrBT5H+39wQ}TG9 zdeJd!jnEC>7OVFQr{7$>r(7D2Qj&Kw3aw_os?3OMzo8&TrQXa}J)P=nzv2JuYlMEB zZ}mbITBwLDkbO7ivE)^uSzMV@JvFjht< z@7HN}^m3c}O4au;U~9luuRUP+{sF^<2MltR_JC17IaPawQ8sC>Fv^PdK$Tqb#GCVz zZ^||CCcE`lmc|V18~Y0%m)*X((!`s_R9GI-zG-tCFLi1JvW?VTD&JIFs^zxK733Ry zFNiFU0r|&)p(g;_cEIif96JD~A8_pg+yTJT2DAhL@AH7K9q@Mmfe;XU8E6j!p$HI; z0?`9NXD1Nr0=i>B{0*SD8%Xp3$vBWY3Z#31%yD2K0c1}Cxg?P91BOz7?IXbcG2|P^ zCzNkQ#TQoa126$5zyz286JP>NfC(@GCcp%k025#WOn?b60m?UAUc}`^TwXN6mKX85 z(!8#8#jh*PgOCRy4?>Q4;Ft$CA)Am*$R=bHvI*IQ6>q{K@=Z18YH+Rw1uP0!6tE_l zMN~%lMuJeHh}JoZXqu;p!J8Fv{8mLo7ARuJB1JSWQN-D$3I!ca(5VrdR}Y&vm*1%! zGZ$x>(3ddu=2~Q$A0o0qM!9y4X0(5y{_j(}nF%$f+A=}3o3R;sW6S)?(%kWHa@WV+ z@5-*-^q(p|^rm&!x{-(8M79)YH+qXvZvB5pq21tnL18%!I0gXcAmGXZ?k@mO4rnAm#+Rn}N6s=-mh;+(2?Okn#ZOZ9t|4 z7-$8uULdy}$oqhy9f0i_!2T@Y*bO+J16+Fm_uhZ2*Lhy}H?C-40!)AjFaajO1egF5 zU;<2l2`~XBzyz286JP>NpyCNoyP z6S4`}gcGa@i)c57Nw(OW4hSZig}ZstQJQAFcCis)abQ$)`OpkpJ@x((=; zuT*7Moq48NFNAfv zscGI#Q_Q+WR^4K`c}cjOqZvf$LAld_T#(~&Tpt~1{9|w99K;k5jOaiGsAe{m-9|8kuAlnb*GC=+`Ff;(z zJ_GEZ1CB2N=U0I1Yry>t;Q1D4`3~^jdka3_+FR;%{`+pJ_@V-S04Bf$m;e)C0!)Aj zFaajO1egF5U;<2l2`~XBzyzq?aB&eA7jbdX1Y2Ch3rq9D(iOk3G!H@^gggj2=Yew` z*o16CHX)mkO~@u>6IQ$li>NnxdJ6Ou=qX^)I!^?i2$ShVD5H8)!znkVPxz5kHEGyT zx%dud(p;?FTz>x=MVwk8N-%G(?#h(n(-b_Xc0|3VR-HgI-nYL~`R4j@9D5VgYm!a1 z{1IOPD12GTKy$y)B0KKh1!V4s~11TSn z-T`F%z`!mb8vt@`Kt2czJrCGk0PHUUj+X%ED}ZYs;C>bG><3z21H7*TzJq}O5D+*F z1m6VOk3hZ&y+!#(aAgM*U;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qq0+eqk-%!4x zd_(z$*EnYrvI*IQY(h36n~+V&CS((`3E6~fLN?(9Yr-P(O%(^la8L}r6M83;$vfe% ziC?pf@=ZNQ-IO}!r%voxv_=R;h>NwG%kM8Qw}`X2B=cs>4VXJr$;LE%Tddw6^EWMc z3u-r4oq4nSYTC`%%)AL6y~Pso{#x#Gq&v&5-E^F;GSAZNni^H|nP%#jEEvosH?BW+ zpNg&n-bfxvR?qX9FRW?4CMjaw}AaSz;W-dq1dbiT=xO)`vK1bK+Af-`yk+Z2=G4)1Reo` zn}GI5fzTEpycLK(26R6D>v~=6iCNfC(@GCcp%k025#WOn?b60Vco% zm;e)C0@QA}u!swbxUgt~EiB@trFm)TieFlq2O$qa9)z6qz)25mLN+0rkWI)YWD~Lp zE8c`fv>P3#eMrK{b^5d}cp~sbm`o008MT{QxFm{bUZRMzOBK;~nPWKe-pyd9xOWpdyl2*k-p!!5VBC$_VwQXC zhEaGo_+oHawgWLA(7gkQ`+?qFKq3Gn+kjLMNIwr`+JS)%AR7X5F9Z27Fcbl7`vLoF zfa7(*c@S_N0^ElI&znHY5y1Nv;5!ER-v$Ej0KpSL`@2BsJs|u(5d8q?{0NAB40L}2 z#7_aeXW-o=KBafVaW_nW2`~XBzyz286JP>NfC(@GCcp%k025#WOrY`!(7T~`L+^&( z4ZRy)+MG?uCS((`3E6~fLN+0rkWI)YWD~Lp*@P3U35$3)R89<3PNYs9$T>eaX7tQ%9cWnwuuyDH<{#0P(E=?nk6-1WvQxw1PqO=pTbH$5j#&mOIF zGfi$gG#cjyUkv_A7D#*nBy&LOERfCvnRCFv5Rh&Bclb3nAioY6vIDjU0Q-8t@gU%Q z2yi_NxE}#Ln}C)_0q+*Tw-xX|1_T}lf=>YLPXeK*fbi2mbSKdH3=n%3=-v&)p96aL z0ExXo@&zFEB9MOR-|KamSN@%s5oH2QfC(@GCcp%k025#WOn?b60Vco%m;e)C0!*Od z3DCLWsv@o`;;NzvwyKDil;$O+D}G679)vsyc@T2K11CJN3E6~fLN+0rkWI)YtauX^ zac+bpIPRvJi}a~n@I;UVPlT#Tb0U<{xvArPo04by)Wb_!BMig0#p?aS(Ki?GDVK($ zjMBSVtQs-X#`<=L`WEzVCY*Oup;)VHb<>y% z%OiR>ZEmC4s{Q?G`1@ALBrMb9$p0xEbz_qIp3-0XM7>3i4+n)MtST%KAQJ@!4glFs zAlC)tW5CcGfbC7degts51vrlZuD1dAJAmf|(DE+eeGl-x5BNU-0v`dvkAe12fY2!* zd6S4^>SQ8e}Zn)5X@+`EcHX#ZAniT2uJP~jrlu^5pIO3+{ z@jf+T^Xg&e=JGqWV`kwj6S@+{-CWVLsa5@$k@TP+{AjKEXRm19)T$4e#^P41{#pGv z-|B@rN2b~$WJ%lVL9#MS{ z1GWZi_1Xi5?;kK+c)%c6X%86XlT)=<7-f_83ZtxO4^+ulm2dun7w9Y|`TTZ@yxOWa z3GF{4_Uo-Gt29c(dyT`-!E?=3Rs&QP^K8v(nW0)O(@oh8zp*r)fPCZJ>qZ`W)7*TSe502eZ@E;yF(Hgd#}(uod@qPBP9Wb547mW?CcyqE;Mf8 zAh{n%y#}OT2QmkNfkQy{Fpzr_$R7cQ-U4jBfc-e+8%Kijjo``-Ccp%k025#WOn?b6 z0Vco%m;e)C0!)AjFaah|@dPN}aCs4z7jb#f1Y2IjOH1?8(iOk7G!H@^gggj2>4B3T z*o16CHX)mkO~@u>6IQ$li^w+yNpQ73SL++8t?)#U1W$y?b0U;czL^i9L=lbmD58Iz zP7ysDfR2qo>o%ZY!mjJEXBKvpEU74y^l#1=gEy}cy7M>8(YoBfxp>d=vbs2UyAu4H zk_?+WRAZ(*=E`FA?~I9FGs2&(r+HJddGl9I&8dc~qslzh@{cs*Mv_-wFmKLp%Vpl! zPW;Bwyz^GM>(j>YxS2PR+`*B}o7P7P%p1MMDEIw;qcCsqyT^ zfahbNL?K=qn)nH4yy<==>IleFt>kdn+uPwLtHEK;nKN z`2dhw52POiG7kX*4+GgpfZQe^|0poD1+Z-c>@9$!6>xe1*LJ||yR}~D*>P*d*A(yr zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@GCP4Fs<_*monm06Wc!_g1A)Am*$R=bH zvI*IQY(h36n~+V&CS(&%uqG^G-f*q`cF5Nle z%o}_$SS-%~o@arU-GKKwz_$nR?*#%c0Kpf5_LqRrD?oT35PcQs+z-TF1G--a;s=4= zLqOs%kbDzJ9RbpB0hwdKz}rCf9UylC$iE8=y$9G*fc-NfC(@GCcp%k025#WOn?b60Vco%DxUz&8?G(l+9Iwk znqX^-cxh=~TDszwmgYgogOCRyCp~b|1DlXd$R=bHvI*IQY{H5+VG;ADN)lYE&!zg+ zbXIsGU}OfJl}U3Vl+nDY=a`#PNBq=5Ov11jMO9Ub`#0zJFE6)+gUK=_XgAkl&s=yd zPLMUCy7b$8^~p4EN-}S*hXhk?G0Eq*Q{@Z&V{khYwTUkvHD2Znd<%_8)TB zbH~q=-Mr~IU1gr72{ttlY^>8v{gMT9x#Y(6$8H>*d87Uct&RFom^b)huvoqZ{NDh9 zZ-L-NM_0n27B5WWwH-Vby>0L0b<-46oshk)LPfy5&~aubkx6i9CYGFyRx$AIkP zK<)`3|0FQ<6kyu{*!_TG7vKy4t~S6O1U%0JE$x7}1Mr0a|I0ui30AW)#XyDRRxSt7$dM$6dOC=2Ne}Y2i;W)_X7)2&zi8MaYu2)ycKm zRxjV6ZLdD@W<0%{wySwJ=Y5-UdpE(?{=*V^%e_A=9!u#GhLlRGsUc1WYsN} zo0k;qU1-%UQ}@_iZ5Y4v;(nq}~P6?*W_x)lOY_3g6~C}F4?-SD1q!_!&C973eYcIQcPt+p|E5NW5}cc{9XnH}$}Z-DTaCJBY~T0tzfbMvs^e~U zT}`_gn{hYMM}BMRc^}%%=er*(yLQubrnq*~bK>;u(P=kk%QU&~0mCT9-QbHsVc7)4 z9tFC$0P(Fr?_)sXaUl5wka`kGKLuo-1_pKl*=K;B83J4{ z1MV>3i2yB8zQ@on}l|g=%aQc zxTb>%FaajO1egF5U;<2l2`~XBzyz286JP>NfC*GQ0ctnYZm8W*yP6S4`}gls}KA)9c5HDM9$hO63t;Hq{C0g_;JwAKtf5hmt|P)6;h4myb< zx*8PGcBdjXuU16kJy3Dh>EPmQ06I1Tt=jo^`_8^ zDbQrf_|Mg{%o$+dQy}{pkoz3Se+dkI1=#X{ z{T$#J0-TNYaBOUVdmZ4h11(K}*8%u80DdPBXa<5VpnW3{as%PbK-2?tZUbU1KzAz; z_X54!frJl8?f_DLAib-;UY7~fSA0bQKL8V80!)AjFaajO1egF5U;<2l2`~XBzyz28 z6JP@LZn(0DD~q_YXo9UQ;#H-2Rq2XfRhkDO4?-S<9P+>+4{SmLK*q{%ZApA91Yi0CqB#yZ$dBjf*yrg02=JGqWW2WIO6NVB- z-oSzx*IPZk8~FxvwW-=-xUewLDkbO7ivE)^uSzMV@JvOgf8B-mlZ{ z=;b!`m3oU_HehSORjahDMG+g>b^~Y#l0)=I_s<1o_9);u0604VR~O)p0iHL2mTth?1Nh>A|0odX1%k(c_5=_*351hC zv=8V^0kIE(?lchZ2YNF=;xv#P08)cMItygJ00weE_AHRgL%YeJqjtk_H%x#DFaajO z1egF5U;<2l2`~XBzyz286JP>Npz;Y&yP6S4`}gcGa@i)c4APH3D^y{e*mB?+Dglj=k$qjoc&BW_9_?=xQv-n>Q_&ficg zmzy^i?^#|}3kPpkx_Kk1Ld@UB@{H;I`RW^)&fg^U&NxlonXkUlNVA5<;?a7c*ZdhF zOWIZ^*J@jR$^Oku?fK*E-~1JCs<%|b*in_9YWYXl>MWA{7j0|0u$>~Ww(3nn`_G8S z^;VTt8l~aA#^LASx#lXX0XB!Q|@wq{^7X!H<8@Ik^P&V zZI}5sdO5P-Qvb$mG0Q#nEBH6~UT|5y1M>I&CtRDgfXxorn*fIcaBcuxPQcv^cw9is zM!@R^e47Ek2MBBff-OLMD-iMm;q5@w2XyWLVt$}|7Z48sy=_1u2qd2eQtd#x1IUDc zftP`77|2C{d=wZu0N7p!><9m|UgtRUpS+?d6JP>NfC(@GCcp%k025#WOn?b60Vco% zm;e)C0u@hy{tZ_cafK0A7)`JhM!dQ-uP$Bjt4s4B0qrH7KI(PKA!nc}GVQJsWhx2F}HIFqmd!%jFtvoLZqPLBA=Y+$_7A1#~T%%5qhJ z87JRn_>Jl*-e?nXjPl8;npb0#O`2C@lojoPDmkcHLOO^xs$f%0yjiV1Zj#S$%O&2} z-uO>Tv*&km*H_7NWfyNER~URVcy6m~{;}r&$nbw|nqfBnY30g4TcxEd?pU$%t`%}~ zv++{NMsG37t!s6c?i|s)!54$Y(hWFz0B0O<9R=LIfaf^Sk^sCX0bdgE_W^+v5d08m zPXnQTAe;fBr-9A^AT|heXMy+^KyMC6oCT73AaxE%4*{9R-@&r60oipx&JN_8fFTE9 zdl0Zc1UMcBoR0vmO@RB+-_`3pTYgvZH3j?tOn?b60Vco%m;e)C0!)AjFaajO1egF5 zU;<2l3DCTuc|-Gt<_*moUgexk$R=bHvI*IQY(h36n~+V&CS((`3E6}btO<*lH#%1QcFa*k(>#T8jwUA6K}=dB z5K(dYW>~~oUTzC#aY@?E)u+JJsiKUrjJF!~uhG0QRa*=fM&B6ad5~JFEmiVtZCfqR z)wYs+y|x{nkvF@qX5M^1@}|7zjmz^pOKU&Oo8z&KR`X2FhMBI~FjLIBMONKnxp_&! z-i21(GIejA_S`w@b8j?Hy=Cs*tL83jxP4Va!`}p&j?-1vp_G5tKZNT{s;5q@g z-vvDH0WI$X-VXrZM=)>vAJe=MT-Lz^m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyvCu z0L>e&E#lfDt}U8iYm0bkXlXO zxo?Ah6ccao#o(}H0RL$qFaQJxf%YsA`T_{&faqDEGY`bh0o_ADyz#%_*w}!?Iv{BW zQcXbG0c17+15O~@4CGuuej_mC25egZ`&Pj57~p&ya6JLIp9DNl0WD7h-kpH&8NmN6 z5ZDa_p99+W{8zm$wD-R%zM_C1fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)AjFadfu z^ls?g(7T~`!>gOK3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g?}jVwC(lZIdJ~f1 zuSt`xnkNEIgfee{2z?38&9w_OrZr*|x7F&C>D){> z=Vn)BoSR_#e_0~y{#)+ycn_4_xoJ96+_~vFaeDUXoEwwHEJu!v#<{^4gTK-NghD|0 zWgr>`IwL?V3UnU;;+;Tm7m$bn$v1#hH<0cDGI3zwD3I+1a>s#u0vI|8*xm!|?*onx z0Ov=5>tn$E3E(*ew44FFp8~$m0RQJe;7cI*70~`Q5c&oPe+xvv13K^h?|NNq?SFGQ z0~25ZOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-6SfX)q96>(J&R~1dLRYkm{G%qP# z@k>hcAml;FgOC#*IN^a!$R=bHvI*IQY(h3+#hb8*b0Z|dMfzN%UqxGmCxRq+B21nW zp^VN=9b6JcbTufV?M_8(Uag46d*I=$(hayz2>tMFv3kF7^v%V4%BA5bC3!caP-d>`*VL5e)YOf;cLUR> zUVYQQ{R-oAzh0;|OSMJFlD5^!wc1uM-=J*`7q&)up7!r5dA7E#mgj0)Nxoj&nl5ao z$TQ87NoUc?`*qqKz1*h0Qg6}A25b%3>a_<9-#=iu@PI+C(jG9%C#Pz!Fv=$F6-HUn z9;lM9I_~B#c!3U6U(S!cnIf;Y>P89wq|F-nRq27G+;K{P9H&+^W)0hg&Bda&Tf^j!y%QU&wZ5YM48+NfC(@GCcp%k025#WOn~YQ)f=ieRBx!>@Y3dNLN+0r zkWI)YWD~Lp*@SFDHX)mkO~@vkU`<#=z2Rc}$+Os=>VzctYf`4u^F+XjP)7Ae`Uh5N zMynCkKusD!^r#&(6K9zaC5*fotLZb#Rn=wM<+o+(&aryaEB$49H`ngnocC?Y?cK!V z|841OT_AUz{QC2(F-Jx1E~a%J_%%!z(606O#!(NfqWVm>IZD60Q(uh@hRZ^3~+r8 zxW5EEUjZ#&1Kw`{-?xDOJ0NiH0ys8nf%f}=(EUL80U){_=zI`}Jp^<=48$J+dN%=y zM}g!PAhi`pKenJ=mw9{vmozW|Ccp%k025#WOn?b60Vco%m;e)C0!)AjFaah|@dW7I zaAgr!7I9_K1Y23eD@*gr(iOk5G!H@^gggj2=z)VC*o16CHX)mkO~@u>6IQ$li+DE% zNpPV)7wQ{ntnfsT1W$y?b0U<{yP3~1HzkkwnU4`aYlJv|L#K@3;NYRjKN*00f*@K3YG+JX2FyrbA`9X^L65$f{c`H!msJyU?mzrtYoNo;yc< z?v3WDx6HkJ)!c;*x36kwuu4l;+_7TiT`S}(2{@75!I1@=p1nqMjrRAas#;C8Nnm`D zxOD8zG&nhZN3WpY;A=r;@dBCcz<>|P?f`OrAioP33IMj}0Q(-mu@`W@0JvTR+%Ey1 zSAdp%fcI6vw;%Ao1_WLQf(L>2LqO;-5PlPg9sxSv0%FI2?ze&XJ3#LVAn`7cd=E&y z52QZ;G9Lj09|PG>fZQqQH~BNvZ#edb2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bg zJ^|`C)NiQYP`{ym!%LjA3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g{e}zfC(nX= z>JyUSuSua!#}ff3LK*d&8Ym@-IDV@lA`29;W04}7mnh=wQbqLLriiWvMYP?iP|-QB z=t!bxgKk(nx%dtS(~K;GT%(LrD-0#rHy4bUi$Bg~)h8~?O& z<)8m1>>{1$M12e|M3A4oQ9 zftLFK@BM)90l>c=2s{V`9|GDR211Vj;Y~pFQJ`}R5ZelLKL*4f2YR0X5>Eojr-0Pc zKzb*Tc?KAG7Rc@fa?b(zJ;2ajz}5lSL;q8+bG-aN6~CesSaM!8evd`VPups?%%9Ipb- z{ebH=!2LSlIS8~I0=$O--vbN- z@42FZ2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6fr=+U?S|S7wHs6Hc%uETY|Tq5b4pXisfI68tqO(g~gjI1$RI-PCf% z&8X)4z`H59)r^aVQ^K?xSTL8|YSe!k$MjaG{(Y)9C8;;(2i=U%z?-(KsW<8%937_b zoF9BMMP6;yn}qhC5&QL4l~o$0;l0M;=is^KDysn^i+Q%@w9HVQmgy$f`rlhx-+_A5 zx8tp{t2cw0;_6N0@Sa&CsyEHe#!D?5BPIYvj*dpX!54$Z@*v=O2xxg2@IC_gHUa)e zfxs3ZxD{xB3VZ@9RKi;K9pXo4*+;)SJoVd;urSegeR4?-S< zob$jr4{SmR@OC1^Jz88*vRKc>38w`J-dq;oUjoSR)&b8gPNHsyA1yzl(p z()RQp+9nrpbLr4Q8t*5lvGiA_;?W z^m6kO_$%+I{>ll!|1J=C4+y>ww0{7EJ_5oY1JO@_&Qn0_4AA{45dRG5{TxVq2_(M) zQeOkNfC(@GCcp%k025#WOn?b60Vco%m;e)~cmi~8 z=-kk`p>sp$hL<*H6S4`}gls}KA)Am*$R=bHvI*IQY(h5S1Z%<~&J9=DPo7owbS5Oh zUy~M{fhPh^gfcoeb#O@((bb@cwmTKEd9@-M?}3N2PN#^T4M4|6pmiJ2FJaep*fR?| z<|v|Ro+1WsR>bjJ6%kpW`~!^5smFI;BXq;J#p?aS={FbeDVK($l;quvLaUjtDl?{Q zze(yJp?Wjn)SC*8y9qx12TSBjs5jomwKL5#HHl?9B$k_|n01S+y2a2~3id9v>XxZ{ z>$K<2QJ;IGdFm~5?_M=`VZ-gK8XBz9(iL~CSb5h9`AXw%8dG6;MD?c4T`=y(WHHN) z$Npz;Y&z2V{_E-vEY zq6xORh*y>7Ri!I_RcRiCJP3IZa>xURJg^Dbgls}KA)Am*$R@0K6Bbc#swKg-`dq88 zqqM>kK@vO>CeMjbM)gMe2UclDs}a>eO&a!YF27SdW+u)up)X6eDl& z#o(}f1;oAvy1xP9-vYhg0f~G62*+kEkh%{@-w$LS00!0r*$08#LqPsvVCWIR<_7GW z0fz^0ZUbB`fV&m&c!8GffY%53b^v}q5ZDC-13-Hl5DEg}=YeQD(AfdRLO}P+Ks*fe zM*di@OGN)z@r!x#126$5zyz286JP>NfC(@GCcp%k025#WOn?b60VY82hTaXm8+teN zZg_cfHX)mkO~@u>6S4`}gls}KA)Am*$R=bHPOv5{;@xni{p49`Pj5mJ{55IPRq;f? ziBLxGWE}Hy7_&URDhUZ&!kPGirCHsBW{kWTEDcarbUu z`qZm$df`_Xw>$MhZ?1-7BVf~B&tCw%kwuTE^qdZUhca=O_+g8hSwXGyyuWd~i zwo~MpX33%+H$WzSU)B0h-$eZtXTD9Rj(=dF;D&f~K%7gz8G?x9U#_}4FcpXR{1X729^kE?LCNOXW z$i4;Sjsf|%fuVN*+eyHl1RQ;UGX=Ol1l(!B(+{*{0PktQHvsqtfj|}regU-SfY4bW zoCl)kfX*Qx)~GtXQK(8H0GyzEmklFyGol3nf)2wh+0~25ZOn?b60Vco%m;e)C z0!)AjFaajO1egF5U;-6Sfa(nw7jbbB7Z**i#YMcbG_NdO@heO7Aml;FgOGzBIOu^* z$R=bHvI*IQY(h3+#hb8*dZU*F*XnbvzM!ejKl<>Y|;@{M&Kc(?Xw_0I@z4G^>-~?4)NC)NSFWBm_EmUyS{W^hZW1cqQ^Ce=9fZo(L z#ayGkW~%zpt4#uOmAG^sjuHM%>oy%^mWNfDyzz|iA>%?sGK z0}dbH+yS`!fO{9<2>>l^fHw&Eo(KHxK%fH%hJf~$flwF-M}TM)=sW<#I)UykARYsH z-vAQbK(Yr&#ewuuAkzyB90#%qAa@c1PCiKiM^t=q1wQ~2U;<2l2`~XBzyz286JP>N zfC(@GCcp%k0282qLji{Z4h0+vIK0j|n~+V&CS((`3E6~fLN+0rkWI)YWD~LpCs-2} z5pcNVe)252r$8YI{+d+kj64x=B9u|UsaKs6MRaTgu*zH!{gNVjuEU;L*fB>DP4g5w zI+~wU2R{iOilQnfBdH(D%Pr$9F3G;Rq9b!9cV@Kv@3{1fFIKNfC(@G zCcp%k025#WOn?bgJON5KTvo(oMO;=i!Il;A;?lghbj2?&&4Z8!ArC^%df==FHX)mk zO~@u>6S4`}gcWbXBGQdc5?rOvRr&_%Dm)P+!4qNfoCsxiyt3M3Hxv*uD4 z<#HjXED}x$GjB!`fiC}4*S@Eo){T6FX3-cfY>o0f?cY`MY;9XD&(*e)Jf7A~TV<>p zTVSE3`5dg9`?u!HZru!Kid#34!+U0pXx$8YjYVhJl&;<2dqH7o0~|rX`8?oi2izTi zCj_*-40ywUF9P_ZK;Qrn>;&4ofKUtwzX3$MfzBQv76-bI0`Xp;_c)M90Lha;DhZ_f zfJ_P)_z=jZfm}b3&j3TG0o$j5{WHMvIpF*faD4^1zlL_>`G(pJXWcLXCcp%k025#W zOn?b60Vco%m;e)C0!)AjFoDV^K<$Rw4YeC;H`H!;U2`@en~+V&CS((`3E6~fLN+0r zkWI)YWD`!XCM=@eaH0L=S!hpfLK6HnDbiK*M8JtqM(u{fZAu&LQ!DyzyIi@sc+c{& z(Qj&mp#`&=;>u0WiPN)3tK3w}jqmG9S8njVpsx%8p2q)$USk8i>j0k}@HYVg2N2uNfC(@GCcp%k z025#WOn?b60Vco%sN8T-5f>G4QPBijRKyEP^McY9zo0Y^LLP)X2sz(@^BvfPY(h36 zn~+V&CS(&gog#WR0392F)@?w)gk9HR&n)biqll(?iWt0E5yx*;L}Y>T4=^#O9^ZY9Fbv-o ztM|nSw{lrH$|$v)#R3XW^;qBTP~T#tuRj)#QoE@#OD3JgD4(1Pl}C>|CTtDZD%t~8 za)l<{cw_%-Y1_O=?mC#=S$6H_N|SCHQ(<{T?WWCLFzKe+Vv_rw(wDB?;Cn$~c?0lw z1A!hO7zf&q0-;_Yd>n`-fXj`yr4>1Id0Ml>ySHfy@9fFbHI`K<*15 zp96-@0=91e`?rAOJHUDGA}BU%0r!1?=YF8&0l>Q+@I46l9|8go1Hngt_DzfGb)iQW zaYX|YU;<2l2`~XBzyz286JP>NfC(@GCcp%k025#W6;FWL4YeC;H`H#Z-SDdBY(h36 zn~+V&CS((`3E6~fLN+0rkWI)YoM25@M7!Za`^mG=p4x;Y_-j(6)AK~YiBLxEM*0WV zXhy3L)j&%cc5W`eQ#)oB&N87ZVcN|VJ)2t9k1>tKtycZBG;d7R7K40)Hqgc>&(pS5 z@@#EeEzi}ql02RRZ?0CZ3Q~LKx{kE?E~UFfL=e4*aaj5K&lN$2Z7A2rQ;S zv0cr_aYk*pU1dLF=&vU?vA+#x^NpznxhsQ}zlAh%0 z>C=DSkMlYN_>KVnqd?#|5c~uPodDWS!n_H8M)O7#eN_cN025#WOn?b60Vco%m;e)C z0!)AjFaajO1egF5pn1c!MO<6NwM8YiwutwY=6$7$eqU)`guDoO5pv7}$2_nJ*@SFD zHX)mkO~@uJdJ|?bZ*)e%rTSc|ucx!ZKLn%TAHt944HEuxxC=z%s!0G+`4#DThrW zi^C7dqV+*pY@I2Ky4kY0I7b%U56hyxN*2wJ$+UB>+BrtivGSUDGicykeSoPn*IPF? zKE6#4r$T6R5N~cpr>RurnQ^zVQM@TR@y4fB1sj8;jJ7G9coTj7AxpRIpQQHFZ=B7$ zc++~5kvIL9)=B1{sgrjkOy0S7f?5C2!iA4mjdK>vU$F4;1ybwU4Dm)N;>}vE*{Z&2 zoHDM*Xc88!)G*mRK|S^p<(SE&x9FvY^%__#uPYXdABeR8T>&8eHqa9U60JaA2uQvQ zq}qUVJ1`IiZ2JKFM}T8LQ2R09JP5cB0q!F}{ZYVk9BBLm@SXsgP6EEq0RQJe;0qvl z3J9G6+P(zBXMxB$AbK9?ya2?$2D-ih;+KJ*D==>oOaH0pYYO-Qm;e)C0!)AjFaajO z1egF5U;<2l2`~XBzyz286QFs+wMAT8#I;2wwzi13mgcRci+*cqUWB{|c@c8b11CMO z3E6~fLN+0rkWI)YEP4}WF>lI@f=l(eRNp{ng?|VbnL%geNArh}NAspqkx69HvKqiD zb6NBnWzlg5j!eXX$+D=KCR5H)#iXeala>grEVLQgO~#M&@@?VF%1OJq9ed{5dyR^M zGXj%l`bb_p#Bat-G6b9ZHeGqKzQ9|h(8FUfScGI$_Zq1}-D{=0)xGZ8UN22ke=d_I zse6MoMco^vyVSi&x>Mbc!OupM#-fq-Xw(NwwI-qd)8n|-s<2F@aqy^q z@HKd?xy-7Ap<X0 zI>V}&s~nxGzIL+m+I!4n@1OF-qA4?~9$i#b^_@f0a=OesQGHpta<*}%UZVw_B~`CD zc+ar%%?+MS|6SjAZmH`Pn&{qQ_9YAU&kgf#=>Hym+!2Ueocmb$g z3pm#St``CK2B3Z;;MoK;ZU($tfTouL-z$LsRUoh(2)+h{b^vW}0O2=*$Xh^kC(!v0 z5ZeWG?FQm|fS&h&#QQ+sULg4aklOc8m74TNbZ-O~b}#`Zzyz286JP>NfC(@GCcp%k z025#WOn?b6fubis_lE8b-5a_$bZ>Zvb2cHHkWI)YWD~Lp*@SFDHX)mkO~@u>6P8#L zW^r%0+Me18{VPH3#3=ZO@T2-e$fJ8x!AUnGn(}j_AW~is7bl0AHzSZ{hVWnppPD zkzKoK-M3@nklIc4E`6qAqnA4Vka=tf?FQcq3QH779{>hA0oxJ4eiU#V2Wme7oF@R+ zNx=OXQ2#mL`2uJ>1$fT@OL3xLfH*w_EOQsb!qdC?aX@B=UbCcp%k025#W zOn?b60Vco%m;e)C0!)AjFaah&?S>1BxUh%|i%M)^5$`O`J4+Y+&eFUHc@gp=3E6~fLN+0rkWEwb=YeGqHjq2t^S7ZDP&^{}68N4@nvC%`*Z+8Hqj1WF$U(Uo!MvEEzeUz(r)G0q%9WurqkG;Pe?P68 zVTavp8_v4z2G*H#KLnTQ?m?Pfr@wy2%(~W00CRXhxvj z;EO?F*$6l`0kxX}=N7>AGT?p%sDBmkYzG=&1H3zcrZ)iJn}Gifa6o3wg+(b z0NfC(@GCcp%Wp8&NRYB$twsNGPz z;hoLdgls}KA)Am*$R=bHvI*IQY(h36n~+UdVojJuyWy&K?X9h9rx0KijE>ftj(-TH z`G=54?Pe-;5?QoY$)fo&S*%$si|W5a#aX72MaN2@Wi`;S9_TgV&>c835eFvAqGp;b z`tOs);Rj^V`k?#|Fg2$V-+gcx3?9tNX&y`-OdL5U^#&G9fwslUi43wZ+~~!O_K_c` zcXJm8%or>`>S;I5Ge5U9?EGh`z5mmWyn8n{n|4zj4@yINHx1kL=I?no1`E8K#7Mjw zd@(pIUjpv4K>azua~^2C0C>L!n!W*imjVA3Ah7hG;n+M0gq{N0o(96pfyfFV`V7$d zED(DR=voEDp9gwg01|6~zI8zIMIf~SNN)rNHUYLqz`hl5c!An&fU^m3H3M!RQ2#pM z@dJ%5|6Hl@2L73M5@iBRfC(@GCcp%k025#WOn?b60Vco%m;e)C0!*Oj3DCRY$|9~T z;>w~DTUo@rO7pJLMZc>wFG5~~ya+kufkPhHgls}KA)Am*$R=bH7QG3xcsB;lhcODe z1Deuw2dG4G;tjo?Tl?G0qjzKc2li-2>JXK~OS;~>$#`5bY8uX5q03?94J?>~Zk6-Mi^Mm3`h#!?tBZ z&%5dO+~D1qqz0dM1oLk2#o(~K19*1D(|5O^O5?gc_00B!q#@JB#oKM?&G z=sXC-4gp<9fcR0M=Qxn~1n4^fBu@gV&w%vjz`z%P?KEKT104N8Z4z*P1-Mdx`yx=E z20WL5#sR=v{R@0P8{k_8`0c-_)C6jNQS=oB`~XaV2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?c{yP6S4`}geBI5S-cx6 zCsa=8UX{_kG7A17{HXpA^61@6=Zu@2=le{@e4iykcXdOpWOz3jKhDdmhJ(}PuHL|D zTp~<`*`_Ne(iV8*)SEK1(WJ5HrDJ1Z_Gt0IguM=XS$(2RYS!X_2E$$croO-+m8mZ< zNIzAN8>PRhdy}+S-H(wLTeT*k{?p@qTC2h`mBzuN`oY)WwdOLb&QvqaJjtvT!a4yD zt>aC;<-f4B9)f-2d%Jp~`A+qz@p$UqF=ow7t7eu|H#_6#46A0Ya&)Tt+R4gm?=g?P zf65b!rp%~%bWv56)i`It`~?diUm)GgzG+SE9ooL>Sf63vm@HNfC(@GCcp%k023&B0<>?qzKH9KxW1^w))(>S(!9BJ z(Qhuzi;x!~FG9|G;H(EWA)Am*$R=bHvI*IQMQ_3^_KlWrGw9Qj~GtC>5!BSdtZ!py5s(GUrHd+`sndZ$^ z<0jwcP3X`sEFGJFDYdWKx+d@DP3ukO-t=EuCz*eynmdwW?o2T2A6mHZk?)k9y1ESW z#%viUb??-TVD1gR7%Y|}K=>#SISxcW0Xk0rv6Dd8XF&XOpyvx9aSG@=10=r$QfGnm zIbh&CV7my|(}3d=P&)uPtA7d0#s;{T0rhskQv)NfC(@G zCP4Fs<_*monm06Wc#m^7A)Am*$R=bHvI*IQY(h36n~+V&CS((qSQBP3Z#eNo&>kpR zdw^0Dr6@{KMxnojKZHD*HmQJ8)zo4osFs%`};Ejw&Wig_yKN zAfoK@&7g=gFW(lvRScMwg9o0fv#78_^UwAb|CQ@(6<9fz5%4(1k!H-13Lj*5U{rbju24$ zF5qkfTRuyNsC%t+x4JK_i8sx|c{igo@g{!mmzMtB zv!wQeSKiONchjH9?%lNR+c9xS@1}LF{`-oJuxO>`d$TGwSr5WtIj1-*=YhlppzmuS z`3;b|45Y6B150PYu~`P#?SP{OsC59&m4K@jaMuC#PQbGoXmkPIH9(Ua@T~{@^+2Ej z2zr3fOF&yA5Z($zyg+mt(AfmUnt?7K5Pu!$@dJq#pf3O<-v&~_S(Tb}>nz?(lnF2a zCcp%k025#WOn?b60Vco%m;e)C0!)AjFoB{cK<|d$4ZRzBH}r0JcXKu&n~+V&CS((` z3E6~fLN+0rkWI)YWD}NH6K3&lZe^AN-2u7-g>eU{L>YySLvQE?-ohV39=#imxXEq2 zPlf1yI77I}_;KFo_&4Q3n}c)n!xLz3!GMAOatrUe&9~gXb91Na=9r|r)cqLjjV6sn zBkj?s51@fe!!ug=AUf=I*lX1%bl*OqyY_@mDpQ}(OUK5l@6byo^&NUiR-Y)7wiU;@ zN$#FysjizXwfj$fl6U8(=1g|yrsL@8Nh5P^v>0Ji_h?qVChI}?E4vkcWe<>k4;XkK zu!RBphkzpj)OG;QDBwB(xI2OR!+<9SG#&%IT|iSe;EMzPPk}%W5bOm)383vX5bgsa z{XjGcbbbZIQb5;5Af5(#E&+)Fps#v1KA#OpEd$baV4w!DtpM!L%&ycpo}FFv)dc(i zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l3DCLWsv@o`;;Nz&TUEqcO7oV|MZcvq zFG5~~ya+krffF9sgls}KA)Am*$R=bH7QG3xI5&a<4FwtsH1t~NwcLW&Qp%T29-W)1 za7kp*UL}j>$7Hc)u`H_p4i9IUMiw0_ftJ-k!+N0Ch(mYa$V42NEQ^|Hvgp507Ka~@ zMeBp|KfuhKN__Y0r@PEjCbwjczPbKLzBC*;C+}utV`r}aA~!yU>644AGPOt+OfH`- zzpwdWsW-!pyZI|z9gW2#T^)NfMp|sunuPjKkKXC9 z9G$AZcCzx?d(30+pYp__DKn}bT~t+NHO^Tuf5F1X7f8LQvd_F}*tTrwnK#wDGQ=AL zjH8bJ?2gS%#2b7wNG#oeI}X%;3V3>e#$Ldi0Gdt%zCOU;4+N4x@GBsc0@^MD;WQAr z1Vjgb&gx%5vatbO%Ye8Y=&1n`4xn!(kgNq#bwJt)46FuhF97znfMXp{`y$}n0Jt^+ z?oB}bX27!rXngrsl^X9Wzbg8Y0)7A{zyz286JP>NfC(@GCcp%k025#WOn?b60Vco% zDBf^s5tkNmX;FzSE#keUd2i{W-&>j&AumE+gdFw2Q4efFHX)mkO~@u>6S4`5-h^4i z8(m>mfg1%+q~TlDq5P0mApreny@5+SZ`_@C>ba+CGJyu5li zI9(3zO%^@oMq%dmk4#sT8`IUzs63#4V=`ED(%l#(W3cF@Y3jaAnxyUx(iC-Xl!~I? z{0%49S`1Pd_B!l;ioM2SltyRnjc41hEX`@?H%FSb6imNqP3;|8zv*}*L%%UWziDmJ zW;JcH9)!xWO;K5z0BDf#d-o)d{2z0|PO@b{w#O0ys_pwI>1RXMpQ-!2JbKe+uxN0UEyqyk~)? zbAazW;J*OmORU@R9FE=Jb1RP;bmq>yx^y zmQB`!&{)!n#&QV=3;@CEe}QIW1KO4WVLK410iq6|b0rX~1-j~hxD)7E4J2GZ-x?t4 z22$&RbUiT80N6GG_RWA}3sCzq;Cuyey$ZOu1NE-~o*h8r8-Vvspy@5Zw-fNc0|a&f z!QDV;5774Bzf@|%@Ba&LD#`?y025#WOn?b60Vco%m;e)C0!)AjFaajO1eie46QFv- z#YJ3P#KlD=wz!BlnC1uYUiU?ya*tS?X7f0Td+{l~l#qn-J?f+uwaQ#|pUuQp* zckia+WTl#A_HctY9r-0zoKxjG8wgL!014Nz$qR#=HtA1Uni9P@8qOU06 z2VeqBfC(@GCcp%k025#WOn?b60Vco%m;e)C0!)D34ZRzBH}r1k-SGD2Y(h36n~+V& zCS((`3E6~fLN+0rkWI)YEU_lc;@xNpvk07S3*8nfCsa<@HU%4fl}GQUf@5w*G~x&1 zO~zJU5DzDZfj61TO}Qe*j2qQ$s`C5k-jtks(>$De^H+5yj!C*Y`(})^*s3)N^`G9P zv1qLd%TyW%kLm|sgV&nNtU4$#=1HmzGeNOo#+#z9Ut79Ag?n@9Y+v5poBl+0_oj8< zj)_CMH+6LxGjD|A-aM(v9B*TgS`XZCP%pvnfyd%fJeD;;%nfv{2jcZWPXmzf0DUh3 z$wnZx6-ax5fo*_oJ79keaO?nT-vFF%0q92UB`j=CqT~$xHpNDbZ-O~b}#`Zzyz286JP>N zfC(@GCcp%k025#WOn?b6fubis_lB#BxVng|i%M*D5pONcTT2)H*3!HPc@gp=3E6~fLN+0rkWEo%{)br5&7NbD!*_j>1LkttRcVKT;&%Ib<)vzwo>TfsTeFm zGOBxxRH5#*(%tG_cWtkirl~)dNt4vQL7JlOjnZA}-gIq0274pS7LBw=qdw3|&B{Bq z7OkYiUWdI_eM0x`6S`|p=%h0B3B7b|tojbUWK!Rumt^&cGHKgz(#=)bCg0Lc{L^1s z`hEW@wI4rjpJcvM^>N0-$GLZmSu@kBnI+ZD&Nw>5s+p@CovOZevhvz{%wzAL^2DMk zGpZh4R8?g)&RH;j!NSKENHu4&OE(=yPfr?Fy0NOif1Kh58BGG&_TTq?=f+?$NX?0x zIXC!X@K<_(L@&^n0FtMHR3DJ;2L_UW?L1(=064w|YQF)TmjTxmz`gWe;nzF~c%A|p zp9Z|kfu z1xUUMq_+QSr6&E_zZQKp0Y3l}U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WbZ+R} z(7B;=L+6I~HfIyE3E6~fLN+0rkWI)YWD~Lp*@SFDHerc1VHW2`6lMW9y%u^c)J>?H zux$!9_bQLh%~ZG~vS_c8Me}2_ShH9b)qjVFvrHq4j+H>mY5*f2=;n0XQ7Kv`E)kl+ zZI<#lbNtQqNAjiN$T@j8BhYH5E6R-a_BZ^WFTYw%F2AkvhpFD&2|dPODU5nkqclG9G^KPo+L1{?!rrD)8e^0&9TJ%!K{~d{XgD(b+<#izK2L@UI z+fKm#4&c}Y)b0kHdjQvafct%*elOtp0BGC?cs~M~_5;3;0slcDa0mz<0YXQCw&Otf z6CiQ|h@J#GKLcW)16^ML@l!z086fc`(03L{o&!?nf%FAn;A_Bk39t`9y>V3k8&@?j z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2`~XBQ1k?--f(dd7Z-7HQHd=s;$5YASLveP zRhkzeFG5~~9P+>+4{Sm6DG zjS)Ht+?vU!2)qacHvplHK-(rDycvjW0irJhov#3~SAnkWK>RhJX9tjY1L%7b zNWKN6b^_^lfPr0rEdNfC(@G zCcp%k025#WOn?b60Vco%DBe)Kp?E{_hT;wHZ_XxU6S4`}gls}KA)Am*$R=bHvI*IQ zY{C+2!YtyAt}v^>DYsB=p>sm#gl$u>*;jcKZ>Dq3P0llZrbA2`oD-vtd--nu&5a(; zyu5liI9(3z&8_$|OB8?Rd)us>jhgw&_?bd)KT!APu42r*`5R8Eg=$iUy$<`IVz03n zrO|P3oFDz0rQyZjNbS`dBL#DBT2p(6c5kX1Gu#_x@=)F75xF<`X7E_{1MZK3`h$Sy z5YTu8@E!%4jsw0=0RIUfa1sc9282Eb+P(n7r+~;AAo?ZHc@~JB1G>%w@e4rD*FfSM zpzkt}yaJ?_{sx}Slfb}JfUO3wI{?Q@ptcrp)&VXj;9d>Xy8zD`pwa!CN{x5@Z@8?1 z2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6fubis_lB#BxVng|i%M*D5$`R{drKGn z-qO4Xc@gp=3E6~fLN+0rkWEyh@cTsFe8E|;*_nzh=_S~ zZ^}97CifvfM$xenx=ohpG8jv9qkVJzk=rzJDul>Ey%`k?rb3ZxOgGyql&41JyhHq2 z+Bd^azVT^Q%f=x6^(OX>a$Exm<*J185Wd51@F(iD9dncIn4=r5y zh}Ae}!TbdaA73C<@5-=mOct}$zgzqLaX3RyzQH$x$+7|PZUmY(0lv+Ee+v+J83?`t zgkA;OwgchUfXEIY`UcSXCJ=iI=-LUy-vN4d0g2r}-yR_O9*}w;Nbdy(J^*YHz}^8k zqCo8dz}X474g>BOP=5^YbODXsfHw{_eG2$`0Dmv+n?Qp04JY3)0Vco%m;e)C0!)Aj zFaajO1egF5U;<2l2{3`;CqVm#_6_YD+BdXsc$afFA)Am*$R=bHvI*IQY(h36n~+V& zCS((qSQBQkZ^{a@6r6?&4Ht?h6i?VT1)G4CNBgEy5lUpyvKnYu5A+&k(QyZkOvHi7 zvZ$FRQ_xWbovC8$%q2p|LSz+i1`p25w}~?=C-vs0g3QgtnUNl!uZTA0!EKiEnBI-a zV9`l;LuD~o^wKnSUnWgb_XcT-x;ILNoO-i;IPYe3rr!8Y{l?Pz)*Pulbn$ZDy_=TP zW#);hUQ@1EKI2TiMhn_Xs$OyMo{@PsW_ULpcHKz48+0vv0AS~uWa z54h?9cLPxG0X#1Ojg5eJE70TxeA@tj6A)+yf<7ShI?(2yQ>h8J%qjZ)Joy2b025#W zOn?b60Vco%m;e)C0!)AjFaajO1egF5pm)QSMO<0Ll|?1CvWPdA=FO#xesgJFguDoO z5pvc8XFae9*@SFDHX)mkO~@uJdJ|^xZZxzZXhYD3D3}ew6LHK|VI;&ndN&o2Nn}ws zTNW4R$fEmUS+rNlqWLjdtXV9J>c2z8Sq2-2a?Uj|3F1x0R$dSfCx@9g-zhPJZp^4Y zsHAvvO}Hs7@us;r;!WtSIhKy6e=D_b`t>bto${fX@2P3yiL6NeOU`aKx~Z^l{1N!)oyMX9!pmPrpdk^S(ABgV-dOiRW`+&ZWfaHE4^)Zk>2n-woY%#!o z3~+P-wcUU-4!AxA+&w^jFW^Z4ji&)`AJEhf_>zGCD6X{ zY(VF--&Sg3_TO?%0~25ZOn?b60Vco%m;e)C0!)AjFaajO1egF5U;;%?fZ`3s8;Um+ zZz$gI=H_fdHX)mkO~@u>6S4`}gls}KA)Am*$R;eYCd?w<=nJz7oN^207CI+%PS`dD z8+?^V@rDy_a+~f`0VyfNxXJi&-slK86S4`}ghg+{EY6Lf z2|*KrCPcwZ2%d=3wF)C3=Fz#C$^kby5BI4QEfbdr-QYG$d7L@>=K3S~(s1MvdN;Ea zBW9=%mX~MeE2sF*yP2gtHjVZ+Upa-oz+0uzn+@=5gk)6r8mU6vYjYlUbEo=xlXRE5 zAA`Noq_JqEJsS0aR%)JxXS5crq{Ci^y;glf_w5t9YftE;GW7|)bZo5p4!vYj-=UXe z^@%d6NW*U82Y+kn-!xZh|7_=bdG~H^HteQ49+ZakZkka5Z6&sVqB(=8O z%)7xCgTrzNNE`wBjsnT!K#i zQVnw}HEGXWE@@x_On?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_=n2rfp?5>?hTaXm z8{XHPO~@u>6S4`}gls}KA)Am*$R=bHvI*IQCDw#lyc3Rg!1+hno2A0sWcYE44@ zr^j)vRbiP*WG*L%7~cprsSr%mg*$bo3<^-^RC|Xp2|M* zreWK%p-0}-)#<;l*=RAzrmkA^{ewfhH~41oST+OcEx^FbfNdLKZvq_6K&=mOz7Dwj zfV%~#4*;IGfyN- zrwd4Q1ATEI`6-a<0n)v|KmxFx0qkGGy>XnSd&8MGOn?b60Vco%m;e)C0!)AjFaajO z1egF5U;<2__zBRx;p!r;F5>E<5?fuwTTAoS(nY_uG%rG4guDnj>4B3T*o16CHX)mk zO~@u>6BfM*v$!`p8WA)iXhamuh~SAhWUDYDVjkU_>F`NpQT=yW^e)rLqGKh{vKnYu z5A+&w=nfpAp942(I^3isLUVP)wSJ?1bN!Kdc~x<6x*YVIoD`b*iZGL(u`#oh-$VBX z<8XA>#^2~AWq6IjQYKAO_XcT-x;I`McT?c8H-E#)wH5;glr1mYV&KJzRX-(}N+P!I5mEqp#5k&vHBXV!>&ETX4An65C+kkWvFwhLxb^!J_0LPm^?OT9zC*XSLca<9VuHO~?j-LDgOn?b6 z0Vco%m;e)C0!)AjFaajO1egF5U;<2l3DCWvdqekz?hV}=-s7B2$R=bHvI*IQY(h36 zn~+V&CS((`3E6}t)`VHy8*O10g41uI-$Lz#+6mjHVDqo?=-!lb)=lnnevFEnG-%&k z{{WL|uJ>+ke0-ZG4hGERpx)eyP*bVsGltvPsNR&EdgIfoij6^1#@iH5y>Yhw&eBl( zd#SzV%E}7!o$5rK@tBBn?-;XYrd2abs+*m0bcR(kS2;RWeeGoBwfC6E-aqAuMN?)} zJ-Vo>%4(dmVE%%Ik1vo~Z!-3#|I#|i{4@3alJfouX8l787d~>`8`oy2HwMi6=~%8E z$=DlwGiWR!!2K>z-v)TvfyOZ4{Sau10KN{u9|ZyjfM6#OIt;YMfbcOO(gj4jfzCJ( z`xNNv0ph(tPXb7s2KxGdWIvEf0_m@SffQi-8nAx@I4%RVR{-bI-{bQ=3AmpE>YoNY z%YnufzpvDIpZPskH824tzyz286JP>NfC(@GCcp%k025#WOn?b60VYuN1gPF{aS<06 zadAz+n$;LN+0rkWI)YWD~Lpi{6A;)SEI25fmaQ zL=;Sj;E6bAt1up79@U#lPP)l;%Fk4>b>=v-g=H#@gGcp)ufc20WmX+T7xN_5d6|HR*6}7!?e8tk$DrPP zk$5TZ>P^e(GV?_BY~{+?#+iDJ7L4YSs#hGmXJqP)$zqn8BO_98@XesH)B#>6(6k!x zxd8tfAm9dq>w!=`(AEHiJwW6oAle9YZUtgqplcftZvuLnfrJm}dmTvnfm92S4gdph z1GZg&eK+9P1Ju3;INt|cdjaa?z^AXUvAMkz*G#vzdhXDT(s5gP5RBuGlmsRis zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@Gsy9?`sNPV$p?bqRoU;kpgls}KA)Am* z$R=bHvI*IQY(h36o3O;1FpGMlDa<->>Mhh;Xr0hHVcQgJ`c)p)n+o_OvZ$Lai;Ht) z(fzP2+N)&I{Fp4(ES5#}-(}IeOe53J`PR>Ydy}!17evL$VdhPq$Ir}F6c_Ul+dSpj zJfC#fUr6z$%xr`hqnD12g*T(c0~7W->}BQ?DSGV=!C3=&Hh5a@!1XNP zeh#Q#1$dqZ8eagsYk{V9fbT`XzX1qr1cIA@&}N`*%O5H=;g|owH4RLF2`~XBzyz28 z6JP>NfC(@GCcp%k025#WOn?a#Jpqb0Tw27XMO<1`VoQs7YiZtEy6Cr-=0(VhkQX5* zJ#f+kn~+V&CS((`3E6~f!lE}}7V$<;9fCRpb%=uL5IhlwY!${r%%gY%vBn6m<_=l3 zOq4~#WLeZqlSTi1vN-&JELtCwDdwnBQU#=>4C5x_$9bb8;FJrIgLE^LATxwJljBoU z6~$&;##XKzp?8BpIHfi4X4`Py&FBog33>lu>9`2*=9SQayn8n_XR>=Y9Y;@38ku(^ z;N4V*Zt!l*Qpe7WDL8t#Oc<&IhsCQnEZcx^6A)|sE0O)xeNCbht zRv;MyQttxkHejF~uzdj7_W_QNfZF|l^JBnu5O5y?>W=`Pqd?NfC&^o0eUy|Zs^_6yP6S4`}gls}KA)BzonlOuZ zBMP$!oNf!<7AhxHPS`dD8+?^V?`A58+~hpqrxI?`5+Me+S<2(g`8U@e$(M>F=j7gG ziZL0UOdfw+tSC1EpQ@1J&0=)`kE|#_`T}p2LXV*`28)o4>RuyNsC%t+x4JJa@uo-v zZ=x6fVCioAcd314)9Jj6H#Zx2QymXVLy9*I+w|sgvsMV}SX0e3tHC7jUr~I2+>O}+ z?Z%$r;tXmx_+C(0(m?DI&@}+WtN$H}jSWaF1N!VhvIa;wfb>dWpcb$_2iR8uj^}~e z7Xas4z_ku=zX;TC06ZIk#!Y~CGtjgJ@VyN9UjYKI0>SM-=ry2i2M~S(h`b3z-vT;! z0NfC(@GCcp%k025#W zOn}-A7Z!105f>Jf*uo;-RhoB|F8W=ic@gp=TT-XE~5q0rWisB%cLR&;7AdlV0^lE@@x_ zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_=n2rfp?5>?hTaXm8{XQSO~@u>6S4`} zgls}KA)Am*$R=bHvI*IQCDw#lyc=C%7J<`kq1!^`gvtrqreI^Q^61@6hf5-h>c7jP zcbP^O9V>yB)j-2~px1~)ci_lG9LU*Ek$w*%hSfTkUQ?+w8JCJ=ZF2<`+z?*MJPfbec0vImI12Xwv<#P$MR z9{}-vK+i`&Vn5LLF_1h6qz(b;Bf!8>z}5xWyP@AW;?!?M(U(^6126$5zyz286JP>N zfC(@GCcp%k025#WOn?b60qQqgV8jJRTwqjU3ygSoY2IDB=y#XqMaYYg7a@l|aM%Nz zkWI)YWD~Lp*@SGuqBmg{{YFbEf>H#fh=NHGJQ3$?6~;x(qkdBkr9>8oACN`sgR_-MSGPjnjibI>NN65zq?VmOh+&8W&a6+)APeFG6DbDOIi z(T~zLPx;MMZ*o#^t`58@jC$kKs!EMPRcy3SZZg%ItIAEj)f?N%KU(Tudsu4Ues)LR z)tlCv48G~Vv`#YrOf`TcWpK^}v;Lul3m^GzVoqIMhI*q%AkF(nq~740L1Xz0aC{Eb zegQa70j@KE`%9qyEZ{i@G@b{%7l5X(0pB-(|1uD`0tA;n49(_ApzSFj{4@|*4n$V~ zozDQVXMwKgfcPq)=XoIU0?@Y>NUj4?F9PWez`#bp_7Y%k1RPs|S})+-2Dq9YuGF}j zALgnCCcp%k025#WOn?b60Vco%m;e)C0!)AjFaajO1d5&j)f=ieRBx!>P`%+@&e?=) zLN+0rkWI)YWD~Lp*@SFDHX)mkO;}=0m_@xQE6h4@>Mhh;Xr0hHVcQgJ0#+W?n@SG5 z$#u}rRG3Li1m*!{w{HeboO$`yaAxJ?-3;&7OjS%7Lmszs<+oA2DLM6K`*7;b=uEwF zzV@)C;R@87y5k?@UA<{JU1pxB8aCz1jGb|&UZVxmxuoh92k#k~dZWx9s&i>aGW7=E z3>wQ0!2JeL|0dvh3uxR4c;5kzJZYfu z64V>-0M#2#yw{&>J1kcad8nB7nRuJ zBHmn@H5J4fLU_t~> z#4%fi@euQ<-c)eX&4{M_z`e=X$_t|6nylGvVG4sY`F-!fi5s5eWW{_Bx{s$zR zCxND?0N>Mqe>o6X0R*1`LeBzi&jI08K;(HK`U23o7Kp6_x?Tk08-SjTKw=Znw;4!o z0a7mm=~sY(R{>iyVD|xz*MV9;;A{b00l@t>P#*+5tw3W4@V*N)wE@0%z#smPN=@Ly z|0w$1Joy2b025#WOn?b60Vco%m;e)C0!)AjFaajO1egF5pm;;^hT;vy8;Uo)zd4(b zO~@u>6S4`}gls}KA)Am*$R=bHvI$G93A2bd`ogRNr`$rhh0Y0`6ShskW?$t|ypct( zQ5GF{$f9MUEE*=uqGp;b`tOs);Rj^V`k+iPM-_7_MEAoP#!beL^F~L&DHqxtq?Z#v+`#dvR8BhlXrKJPrc&t52FNu+GOBxxRH5#*(%tG_cWtkirl~)dNt4vQL7JlO zjnZA}-gIq0279APW6?-^H0lGb)U3QyYtc$N>~+{{)hBe{KB2qzgib0`pU_Lk#;Who zOD6RldP!EFD3i7oN4oLt`wvU&x<5(n(bvDpyL3}?CcAXgarE@0kx4gZ%Q&fHgKi|# zZt%?@uzUmr_5;C>fzUyq?GO+?0z{4i(c?hpCqV23&~*}se+Kk?4kW$+`c47KGeGJ~ zAbl1XI0x8Lfc+xiNCUN(0OtVUs{RuM8yir+4Di^2#u~uu0Gd_;zFNRv2Lzlza5WHe z0c~skRH+HO|HSnSOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l2^2j6N;h0q#AQWX zR#alkig;6L-c-8iHO~@u>6S4`}glxj1H(?g(Mo@*I3PBa3 zU@8Pp#PM2%F%a`8-Asi{B8&DaSu{T;i#3a7QT=y_ILkCJaaIB?tAU2~0Og!Yh)GL? zc5s`eJkA_`bN!Kgc{p-T;!QpW&y*{E%eY~;smg2edEQ)p8{L~>-J4+t-uxA^j>ck= zt`5E#BQ3USO+x*r$8oJyVVO$f;8FeHYw%ignN?@1nP#43)(T;rfQQ!crqH@SSvtOe zd$Xfq)&FC@Q+;YYp1OC8Su@kBnI+ZD&Nw>5s+p@CovOZevhvz{%wzAL^2DMkGpZh4 zR8?g)&RH;j!NSKENH-gJQymXVL%TP=jDa@-?oIWD8wTEtlNwwZLXO^|mr#E2Sk@^X z%ZosG0}$B=L^lDQn}OICpzCEI{tD3ZDv;O?^t}cocL1q3fb^Tdz*~UrZNMG`9IZfY z2yng&xY_`BJ5V16JRbs$5y0C4G(`d50l?o01P%ki7!W!Jv~>aDZXgl|qMrhtJ#cSg zy>xFx(brY*126$5zyz286JP>NfC(@GCcp%k025#WOn?b60lGJIZ|L68y`g)6S4`}gls}KA)BzonlOudV<^l*aQZFuTd18-J7L=tZ17bc-5VnZ z-Q+yvryOq5pm=lRgNjj8apnq?LXnzECU>Nptc5ZIsn&7z+DT}*8v_U(6}1#x`3uNfX@y1*8_oiAlLwe zJV4t^K)4ZzYz3lTpmQ4#YXZ8Ofw&LodHp{tH3|QJa!CUdU;<2l2`~XBzyz286JP>N zfC(@GCcp%k025#WMNfd<4ObR%Wf4~vmDtK6-ddWsmM;3OrFjwZBIHHLNe`U#z$Ro9 zvI*IQY(h36o3Q9jn8mx%(T1Q6K^vlAHUv+^AzOu!5cBBWOy``NoM-$@7yb7Q>L>W0 z>y7Ho^+)FARl~vQa!_xw*fE12IFApeU-#Ydzu2s{&Zr)&{9)QRcVg0v0h4C#Qukx9 zH)6JoM%trMA84iKX?R8pGen2I4tuTogznoXbl0BHNx6@``J4JH4S0w0D-F_5vDa9P z(&&u6iNE=umj0*aN$uw^gbHTgw5IkBZQpdfkzwB`8DrGW1l7p-iDKlKOhel@_+l_w z-U1RkfxdTu^C~sz zW%G)@u7Dqa2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b60opgTZ)o4pzM*}?JDjr# z*@SFDHX)mkO~@u>6S4`}gls}KA)BzonlOufqb@t={y!Ek*{_t<`-0;Lz#~z8N%@r-AfxU|;odN15rQF*#g7@K-b$q zJP7o(0*MgN_b!lZ15)ikIt&ba2-rRX?E9hKI6kI&Be<@E2`~XBzyz286JP>NfC(@G zCcp%k025#WOn?a#JprmWTwKJ(MO<7|VvCD-e`(%dy6E?p=0(VhkQX7xJ#gFun~+V& zCS((`3E6~f!lE}}7WJl#LIi~f3K0bpB6uQB+A55Pm`C-dQn5*7(Xtw7SP%3XWzlg5 zj!eXX$+D=KCezMQ&7`R?la>gbEOc4zo56$gt{V%JRcsllhBGTS??zTc7=6LE#mc$H z6?prBPQ1A~>83CfZ?+f5yRjXZXQ}i2S!#dtqh#K_o0ijM=839aQ?3lD8E5J>TF_Qf z^@@Y{jLf^yLqzI7J|gc1-wX~*C*U{?)W!hkF~HRYxVwS+INNfC(@GCcp%k025#WOn?b60Vco% zm;e)C0`zX^-O#(CcSG-nw>M`KvI*IQY(h36n~+V&CS((`3E6~fLN;NEHDMO-MpKwY z;B;H)woo~ta>BML*yyV~dN&mub2Fk5KM-#+w(^2_I5`Zw$y9F26*0yzs@qiM_tU+} zY35BK-J9m&+#6*Uj>ck=uFk$0BQ3USO+x*r$8oJyVVO$f;8FeHYw%ignNE(|UYSEP#;loX)y$IWW@j9oVb#o4j!soyJ6U<{ zJ?63ZPkCa|lo?fzE~=`!Zgfo|yL;2RZ^y(T-J80)jF~qEeDMyCb|f=z@Xg?{yac!# zf%>h0#|t!W1H4T@Q#0W60shy4fFB690HFZT_BId>0+CiA8Ui}s1!8SLS33|713e!C zi3rfw0VJb9>Hv`L1O^TRwxfXkINNfC(@GCcp%k025#WOn?b60Vco%ik|@88?G+m>LRW#DzVi?ytOoMEnW0m zOY<4Z8+)Xs%(>Q z>BiIlXG?SNzohoB8$EfKZfeeCmu@BeL+OU<8;NV>r{gTRsmyk7xLDZqCT z@TY;mB_KEegsT4wf{hIbF9RZWAX)=-I)KCf zfNdjS-vl@|1GQTK=gWZW6~O%}P`@4UyaqJx0K9JiO>Y9ew*ddn|Ekmk-uW-yOOy#P z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`8CqU_j(ha2>N;i~lcyn_$A)Am*$R=bH zvI*IQY(h36n~+V&CS((qSQBQEZbV^LfKzOt*h1fgz6sl=V1uvnDBVnjOd^Z+Dp@o? zCW|$TWl{Zih&anMvglX|w5$di)&spp9J&KXCgQ+kS=3CEMgM)WIQ)PtS|61E0S4z( z;=3;qn!#AOVi8mwgYNjjBjQRFAqw-7W-jtksQ>1}6zTkgZT30?I zwa1)+yt_C3HyL}135 zJ)&~BNrU3ejSnhDO~sijv^fmDxgD40R{WXDf?GH6eAJWbmxzLHl|pYeK(G;#QQd2# z3U#lQ?pF7@YkR#kP5rq{nxyUx(iC-Xla`Q6FffX62n)i&oNM zuftxeKB4>e3Ej0PbW)l6gkCx}R(*$FGO6#-OS1YznKZ0;Gde?WLMtD!bi^Ru>>Svi zck!n8RQ91a4cnFtJ@lq^ZH9OwAl_7CF3<$CRtW1)Of}Q2CR5gfdW%77ZqYzuS*b`Y zwLrKIh&X}hYM|2v#MS^^ZXmuM=&1)14M3j zj!?W2T-Ct@m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyyk(0L2@MHxzFu-cY>Z4bItw zY(h36n~+V&CS((`3E6~fLN+0rkWE-(O_)Wz(G_MDIOP_~Ep$%koUm;QHvB4&;>~mp zy2*LS&vXp=St7JoH~h!G(Y?9;$h^FII5=Gn?#-?EGdE7HnWY#tBA0Ex^0K0wG(Y0a zEJffk=8@%)+(ueed3FPxi_t; zy+gY<)s1?yRn=_9DVmKDnvGWKwwlY;V`G(L1{@QT_`Z!}P{f-A#2b7qNG!*ISQpUM z4aDO>&!<452k7esk_jMn8c6p61O0&Q9AG~WI4%ISUjxo>0M})}eFdmrS_R4GNucp5 z!22}Nv>foQ0Q}DYfoFl>b3kYn(DpnKegTNA1)}SK&KH5$2B2#r5Z?s!Y_6)*B(_u) zeMtd7025#WOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-3xxU`5%i@3C?#FiHE#?rj8 zbkT1t&5Mv1AumGCdElG}HX)mkO~@u>6S4`}ghg+{EaHupIs|nH>JSCfA$TH=*eZ;L zm`CxZ95RV44nH7^)(2&=b*3!pX3OH@99eWfEQ|IkSu{T;Q_i_6=NLuDO3k2pbNvI% zq`6+Yx$*IBYB&`_ocj82n(??o`QoE;-XVT%rE$r?$0SWq_oF-Z#-~*k8-w&$?6puxGSwU9xW;0-rr?Z`7F)F@q5jk3xYnw$Or>$~ zsDAJ@c&)k2s)Mv*o}~IK6BK`CyeYo5%F>^LdUN^U(#hsK)u+bese8wmH8ZW6SyJ8X zjH5HGnz_o+sp@MdE3duBJof%6Pb``;qw3K`RaI8woCWh2EPQ-{)OwS#H~p8^N#>ua z@0XPKPcZ8rTDb6$@7`FwD?`1}TJ%!u+}zb0d@X1!ULdgz=xYL!%|OZrq+bUH{DAE( zz`hf3yaUwk0-U=6*B-$A9#H>2;Mof_egJs)0ZktPzWspzV<2!42p$4LM}W4YK=?Qi z`2>ib06I?svCn|6&w=4aR%u75=fo}QsNfC(@G zCcp%k025#WOn?b60VYuV1gPFny`g$T^@i#V?{Cf~WD~Lp*@SFDHX)mkO~@u>6S4`} zglxhRYr-t*O<7^qfm3gx-a_ky)(P9DV6(6CsNPg6Hi;}+Rs#*|fnK95I_|)ci8wG> z7B$mk+BvG3G!ynJgovvTrouFc}fP-F7=<9y|NeK%`Hofkvt26J$9 z(%q^|qnDgx;IEu)V)zEq;#`=IO%3|=G`PyRhH`a{#$DA+PyaK(oM_hGV?^$ zttnR&pK+#MqXlgxRj)XB&#=NfC(@GCcp%k025#WOn?b60Vco%ik<+i8?Gzjx+1PCDzSA%ystFxD_!*a zO7kM*MaYYgV;(r>flbIJWD~Lp*@SFDHeu15FpG7gp$kD5f-XeCTnL_s6SfMYAm-7! zso;>C5e@jcaSTg&K`fjc#@&1;#9S}OjNao)MZuwVlaqFHm#W_sX4Xw}akLxTd;e{z zTlJ{azU$T3@~++VC$eidt^0ON98$aK_vp>v({2nH{?nWofp&xM1%>5(z_Ayo{Qz+8 z16&^g?)^ah$AIS`(0B;&9s!z;0>0yb{}Ui^0tlW2LZ1O`p9A49fXFEzdIsqH5{R7z zy3PUd^FYr9An`TO_YIJ|45Y3A>7|cCv3U}(*#UbE;BWx7D*NfC(@GCcp%k025#WOn?bcyP6S4`}geBI5S+pB{Vb*|CZK2vie2tQaK%!o03y*wiQRYajtsQ(r^OGO~cx~c~@>~&SY0^ zI*y*6G*acpB-LRUP`Nt(W~?&)#$XZ_A&IPN4)m2(ioWtZ;C=z9UkiBF0gW#L-VH$0 zM!>fT@NWhJTY%upK_W-H) zfb{#oz+S-iAz+UHjt-zU3OEk{u1>&x7^sf{o?}2`7nB=sHNfC(@GCcp%k025#WOrYoqP`TlvA}%W8qM{O8RKy!f^M=wzzo9fQLSBTt2sz(@ z^BvfPY(h36n~+V&CS(&9y$Q1@H-aJrMF@%z1rs57B97K7jDMI%c835eFvAqGp;b`tOs);Rj^V`k?#|FfpeR z-+hVD4Q{iP$3uB1`Lb~25o$NH6d}em+S`2P6hnRe(Rj8}Ib%WXBAgC_kvgeHNJe$9 zkt)=^R=Qi=>#ptf(lqtwGHH^!H%L>|y;1rHm~`Vg@u;PF`(LE?rB9v6yLNN4NjKH; zpfsd*)9lKabYrrZrT%jx&~EU(ps<_-yq^J0p98)x0RJf|0xk8h}w40lH zHWi8=V;+sILix2cZ%S_9P4jRAZ~h7$NMkWci`D%YX|Yvn66!xaj%%$7%TyW%kLm|s zgV&nNtU64JF;7y5#!OI##*8=lw*SS_dKuqWgx&<&-U7lqfyg^RbQjRM8;I=zy50lg?*l!1fy4(u z-##Gu5s=ysq(24*4g$8rfIS8{jsdk@fU_HL#R2!HKz$G3=>-}SfcG@e)Cc(b0e=z* zd<6tkKNfC(@GCcp%k025#WOn?b60Vco%m_YFppn1c! zMO<6NwM8YiwutwY=6$7$eqU)`guDoO5pv7}$2_nJ*@SFDHX)mkO~@uJdJ|?bZ*=q_ z=tIzlD3}ky6LG>;VKl@%nm5xqbdVyk^|w+q?-~{)?sKga47*_Z+L6 zV!l(Aa>hf-xp$0NGt;V>CDqN&I6A|snX4R~s=juz^4fdMWAC5x#G)xPsvccbRb@5K zSulUW!p9d#H#2WqQ+tOtZyHu*m^T9EP4x+l*{Ys&oMH_bO~Rs;I{sht1ohZYlw&6R zXEjL82@N!s%ZkQw1qd(wA80mD0@0^{&ZmLca-eGk5Pt^fc@{`K2lTB1lFtLF7l8Cy zU|=0!s|V~2fWrgSz63ZM0oPW*?FH($0iGtHu^I6CfTq_0pC9nI0D%Ayd>aS_fwop4 z90DTm0?{_0v;BW6HL>vj@P499fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)Aj6g>f| zH&kz^-cY@vdczx>vkBRRY(h36n~+V&CS((`3E6~fLN+0ru*8}$i+ZCi%sO!DE!10R zozOaA+Z1g0RUXxwa_A(oIQ)PtS|60f)|s-Xn=Olrb7ax|uq@iEWYPSXOg-nSo?{dp zE3cV1g9^^o2N+9py>@fs zdyQ0~?zPh0>Rxwkua~B&KbJ|9)V)EPqVA2-UFzN>-Kp-!;AbNS%V?xM8ufuzYF6H< zg#e<%UWdI_eM0x`6S`|p=-|;PZ`Di3#;WhoOD6RldP!EFD3g4{&Ahof>n7jkP4t8R zv2?HfU#b12_N{q0Z(46M^QQmOI?4PqRhW?!b7z8C|IosPk9?=>w64|PDA_bj{{Ar~ zcNjp~#T$GtNG$t+*hfIuejxra&~p$-90K}|0Li03>Nt@81Q<90*ggg9J%FPZs7(OQ z(}1fFaQ6fCNx<_J(3k?e7lEcU;JXC)2Y^8J|3b2{0ik6;n;i((01*ceT?us70D<+_1^aT%bzA8a%)=uu(12y@OTan&vTxc&5SRc(>K0ZjQhnIe03aNIdhoV9cLbH zVBhSUTixDnYQRte)>1G+z*3`BtXOK0TBC+iw3aBfMyMFD)Tp%vs5NT90QFqo1-lzc zXj55jy78JFKP#-HP1e=5uKnisS>Gb+5Y!>4Lrj_u!4Kk)t;w+vO%!j+#3j*H%O;?1 z8_@66Ro5-pGXpzj>8f(BPB+K!lIFlmf-9nnE|OcnxwwCk*%HozysR4&M$h2-9hVF< z@7J3%zgoGmmdS@ss&#{pyMFQJqHr@g;?3@{#G6N8>R3Exi#N$F|JyV84aA!#UvD*C zylFXA73+@NIX>~m3TLU}Pva18a9ogBwgRc=fPw8m`gtJp z0+8JS47~(+8vx%fz~2Z|H32nYAn-a6YzAsufKUXee;a6s0*$RexD9B27l^b2(GH+3 z2DE<#e_E+8HUx(@@1ZlLEVkW2u5$AMH2FwhI6lR)Mxh&S0jiZ_a@I+y?xU;<2l z2`~XBzyz286JP>NfC(@GCcp%kK*1BBcti1q;tjS<`8czld}q(atq}aIwy2a*fx_k`N~A`rj%1|#x>#RYB8mFQbe3Q z=H2{ym*wh%!k&e4T=r2Oi{w?4KXlArNbP3Kc{fdCX*Uxy?e&rk+yT_sR?zD`Se@?GO;ESPaL;3pA_-8ht>x5@_}Vk@Y~d3TUea+G~K=CZIC_#5V)oK_IaW z=&1#gbwFPTNWBOQ)C1|AK&Ao6?gEAy0q<*oZ_nZ~i+}Ipg0Cpx7hnQRfC(@GCcp%k z025#WOn?b60Vco%m;e)C0!)D34ObR%Wf4~vO|g|lystFxD_!vWO7kS-Nyw9sV;(r> zflbIJWD~Lp*@SFDHetb=Fo$hGiHiq549VY!(-F#zyKKs=BUgywnf@ZW9!gn z8(XJ#yRn^|X*avZ(rzYZ+Ksn)v8Q_VziA!suNyR7yQw^#Tf6Bxc53GMv>OM;{Is4P zk9LCtgTm4R_#;5o+dxee2($vhHlX%hAk+@jcK{7Bpz%W>+zB*y0g*TmJq)yU1MNqF zSOVxg4#aza?p`2~1bV&#l6^p5KaffR1E+xW0FW63vT0!GN5Fdq@cj(<&jD2zfSO-` zz!U#gW(ltPH?C-40!)AjFaajO1egF5U;<2l2`~XBzyz286JP>Npx_BmyP6S4`}gj1{ub7(hea@K%TZK2vic4@pIb}HZl}Zgi7Rd9_#1pej96c}fW~3K$wJBrPo)u!;xOC+j{qQH^ z)qm1)H^!sPzqxO7Ta_xg*;P6Bu4!(|eI=F!TJ`-S_RcS{ zER?--jQ7rx_ulEg>F(K&E}uQW{GsLL-c8-E5#w%L z;@z}Z$1&~(2L^}52LvmDT0an457buy4b?zn4G`W0GzWmlW*`~_+O`4hwLq*6=nMhz z7lH12Ah8qZX#kSDfWAf`)dUQLf%NM@rWwe#07DVL`wrmS2l)2`RR@5Y_kh6rK=2?? z`vDL-1k`^7@225ndN-=zYby8!m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28y&JA9 z;>sefESh2~i+ERQ-c`Ebca`Qz$diyKA%{G0$OD^@O~@u>6S4`}glxisH(?I%#z7l` zHUw>mNwXpNL7cBOITE6Y-i`A&?9q(ZAu5KKG%Vd*{iJllG@OOXmdD5&STK|ND5H0C zY0k~mns-wm?3n#{D*+5kARwwfxss~@CZ=*DG>S$s6PfY zd=5080K#7Y&0hkMuYu?{K-;%K`$^b0vD36~IQ51JFaajO1egF5U;<2l2`~XBzyz28 z6JP>NfC&^n0opfQU&QrATwgTB))(>S(!9BJ!EY|jlaMDNPeRUm;H(EWA)Am*$R=bH zvI*IQ1#iL}_KlTJ1f2*v5tHUb@PjyJYjRXX6YZN~7$v$odXKJJ@72}L`*c-(zpltEMG74V{aIj#G85w+#C?mp{Q=n#;|btDj$|i&LsBdB`_8gqwvT$4sA~ z$0GU7G;drE&(xZG6Sf*gjl(c(te&gQn~TPc+2&2#z!Fc_+e@{M4)hzR=53+%a;AA#QDI|#Y>kxyi}{M9%0@nm^T#xi~CmNhyF#ZA(tUA z)vX$_(}C!kQWj_|1ER4E0A~d4e?LhtWK*I|_;|?JF643lI5P1cNz6!MM z2HIZ(Vtatjy+Hg8p!?0GWtPNSOACHWPksR=zyz286JP>NfC(@GCcp%k025#WOn?b6 z0Vco%sNPV$p?X90hUyJ(a?U1X6S4`}gls}KA)Am*$R=bHvI*IQY{DtlggMlkqRCkY zPQ8VC3#}7cCv2NZ8-Qh^dQ&DgiLP2U0d?Deey6UwZo!@z*fC32m2-94Ifj`u2WHYr zg@|(7H^U~*BC|D|1$lWlWBWC8#FSAcK8nYh?Q>DgnIn3R-NZ++{5HoVKgyH?LW{$r zG^eq(Xr;#1s@-91^S5tqH{L%@yWN;kW6=&+j3-vDX)a!|daRlaTN}1k;|1H_U$9+z z!G*iVnF&sAkztCyMUoM;5`EPJ_Y=r0aeF00g>;4=np{KPeA(_AoerRc@BtQ0J?tx5>G6{_ge)dp9K2WEGx65)-L0^1}4A+ zm;e)C0!)AjFaajO1egF5U;<2l2`~XBzyu1O0OcDlFXHkdE-#v5%Zqq>Y2IGC;J26N zNyw9sCn2XjaM}Z#kWI)YWD~Lp*@SGuf;V9f`Nl#ef=UFHh)Gi-_(2@BH901tiSkV; zXWfiz&JXOH5s%_Yk#X{vdNV$)W<`GD%qVkhbhDkBeDs{Me@^e_Hp91Z!M72AXJWmZ zrm?)6iJ5wn^e^)a9))-FRLw7@dpCor+}=&=p*=H3^=?|X*xgrnH&!?{t$)5!u^HXF z!GXbHSr4SDfPrctT?1q`0oedBv>EVj1$@r|{_Q~3^FYlDKwt+Dd;@WN1HyZN=Dk4V4Iug^(DoM4{tgh^2XyWS;s=24_khIvK+i!S`2o;(2uOVd415fv zKLIjF;N4_DrFWwWzNUg-fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)Aj(7T~`L+^&( z4ZR!Q;G9j!CS((`3E6~fLN+0rkWI)YWD~Lp*@RQ933GTi_Q_cUPPc__3zZWpCv2NZ z8-8V?cf%<+`AzsKRlN_65N<~NbJ4{3H^s`DhjW9eGQa-uD+HPqrVg>mN4UXz=-iAs z>te$H{Ct^(dDz}mp`<;yu8Huz|zG_mp!~xt2~|Cx#>D~YUY^E&8^1o{}-8c<8s+O z4y|tQh`BbSOE)+&2rLO8dmI?*0lZ%TzApj)*Fe=bK+U&6;3N<{4b*-IguVyre*hYO z0vgW%;h%x#b3o(*5d8&cd*aU!Y*qoWCxOm2KzuFGy$(n`1@t@(BsTzk8-dg_z`(OW zdJB-*3S^%HhPDIV5a4_9&t(>W{hzs>feA1HCcp%k025#WOn?b60Vco%m;e)C0!)Aj zFoA+6K` zW&CHyert)a$e?q0uYLGEc(1#t#0HbaJ=2g{ZWXEJW|#M+KYOZwf_wAPw~eN|H`khX zQ<02nqq{d@`<1h8960cb?6}+;92q>8mjVAPK-H^2&2Av@8W7wA)b0gBZvgdg0u65g zjqd>AeL(YmAaVeRz6Z3u540ZyVjlpVhk*D;K=;Q$;uE0f2$1{~==%&v9RmhF2ht~i z%ojlROJL}0z}pY_Qh@&yP&EM53<7~P+?(K!bZocz`zSYdIyks3CO+-47~z) z8v$Pv;12^;uLCvBK%fN(Mu6J4flw5vZ~gZ&OGDeg7yNdf`~pmX2`~XBzyz286JP>N zfC(@GCcp%k025#WOn?bcyy4O!E-m8HqA9kti1(J}y`>9&Z)u){JPCOca?}GyJ+KMc zgls}KA)Am*$R;d!6Xp`NtsaL~gslzR-(hR$vOvvwATR_3 zE0#mG@dBaMK)nxWs014QKzKdST(!K+5~*I!l?_aQ2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?a#JOS!A)NiQYP`{ym!+V^w3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEiZx*l z{l+>uYr(0wP;sGoLi2=eGimd$Ow@0Rp_J(A=smh>y;oN|@6%QF{kl5)fUbHU)Ky2h zu9}wURCF#XI!@KK-ZCuUT>b=;X-3#LBmQ}vGES+oXe=9n$TL9nS)gqT(7qLjJqL7d2jb5I-7f%%9YD`Z zK=Ngv?-d~RDlo7cNWTVT_5j(vz|b3jHv;(H2K-T=suied0|M^?!FHgw0|>={`VWDI zPN1<12*-iu!$71Ph#m#n56S4`} zglxisH(?I-rielWg$N1}lO{y)gE(qyay)tfT0Np#h+38>o!^gDIcbqn^)z>ZnE zs+_CS&N0lSIWUt}Dx0osmrFNSKVM|FhO;0)??x99X8NQ(R>)^@ zkrQtm+TiDwD|b3DB&aE6fx>b^6qYZ5*q1=(*FgLmp!-`OaT4e`4J5w<`o0HJKL7(i z0qHY9=4T*#4j8%scq{$_#l{QxR|8c(pr#TC_<`VhptcGKRRi@kK*J`WF#v=&1IN zfC(@GCcp#=o&dEQYB$twsNGPz;oZ&Igls}KA)Am*$R=bHvI*IQY(h36n~+U7#hNgO zc4L{GHQ-cRsJ75Jp>e{tnY77QCTcgOoN_a+2|w^|Mm&lqMZ?Ks-pxqkrdYff4C0x< z#~k_PBh{N?+2fed$D9fN{W-sUnR2)tuxym(G`1G4)Yw|{o_cee@qU+fyRn^ytNfC(@GCcp%k025#WOn?b60Vco%m;e(fd;+v@xW0(%i@3gMimfl=y`_0?>4M)| znkOMoLY{;i^}taNY(h36n~+V&CS((`2@BqYIqVxdod`M+bRs6riQoru%GTtlh$h-M zoN|-jgr8E?```%eX2d@iO^k(8tSot$H=`LgV-De2sjTCDER-LaSi|P(-w*xe>YWq4 z=by{D*)^7Pv*MC-b8%~ybCW#x7ti3q|Ij*~8GOfd=ce*>Zs(@!*r}Q0b8f62yH@dd zPQ51QPWUV5#9z4pq<#Sgp7;;=HLHNklR$P2Ftir%t_OTofWI23ssU;?0f7Jz+ziwP zfzURfz7}Yx0~$j>_(h<(9*FD&q76XXE}*>;h&2J7VIclG(A^9qT7aGikbE2Hivp=u zV4w|1zYApA|D(*3?f8#^-^-I8f(Bt_J_0tE2bms`Xy|H!wM;49C7wS%x1A9Pk|m{M|s+QJ^LP1dap29-y`t2ql5~ zuYiU=ps^nar-0^DKx6=j4gzgyp#4W6mH|4?0`V-+eI7^*0X-Ef;MsVAzSTg=2MkmK zX+Mxz4`i!=p=!Xp5%4{;qRirdb_JI;FaajO1egF5U;<2l2`~XBzyz286JP>NfC(@G zCQ$GM=-zO35my&+bL;ZWrs80_Odd0D zuE(YMHG^ad<-lBR<4p9P>z{dZJI2g7JTC3xb{e)$m&Iez4p@vQR;_6+-eL7vH5;}z zY^}x%w!gn%yYzxhyMFQJqHyEV%oT6En^$6S4`}gls}K;S_7a9O8{_a#n#;ZlT;l z=Y-A)+h)>+UzsT0%;TV&yodbE!;qhqGA>3Q_+ww~-dw(Ck*OXIJ}wXU=GXW$S6eF! z6n>8@pT}Z(n>inv|L-Z3@ME|$N^=@pi&koEt=b*N)^=%Y*TkT4c#5={#@3`Ntr+K0gslzR-(hRY&+p1K_&92J1 zcTIC!?klk@(5mkrv3GunWufeyW4w2ky!TG`O?S_JbouQ0z`Gsr zJrDR_0IGHXH7@~ymx16bK<%qQXg5&*+JBZ=8ut7rmo+c}Ccp%k025#WOn?b60Vco% zm;e)C0!)AjFaah|@C4}IaCH$^7jbpb6kA=ydrR})(gnY_G*3dFgggm3>Vcyk*o16C zHX)mkO~@u>6BfJ)bGSEH9?ZEnMO1^R2Ju3m8gwJ6K_e59+F;Tvts?box0L{T!$2T5lP)Z!Uj=u{4)^H&;KuP7|k8sXWx1ynLB_ zoSMtuQ!0Lq`&y4ud5QK7Y@4ZN--NA(W#iBuxrTis`z;s%WA5ae(Ch!{Y4Sd-b!^%G zoay#W>oq3d44&VrxqoliJDS)#x4P|rT(;~_|Il_S_KmP_+@9%LMYU}_lW%ZjFj<;` zh8CbP0)*cNnxjCZ6^OP0ZSMl@?Le#p=!^mJ4}tDZAkhW%#DU~tpsyQ99R&swK>9e4 z=>f96z)%wKehv7(0sP+rRVRU((?H-mAox8{`vVaA38+5NfC(@GCcp%k025#WOn?b60Vco%m;e)?eM9?(_6_YD+Bdw*Ih&A8 z$R=bHvI*IQY(h36n~+V&CS((`38z>S=CE%lo+yeZ6i+Cg+(_aH|C;z~nrPpYiBO`e zmQ6t2HlW|BtFBwHX9jl6(pBYLoq~=bC(VJJv{ETusmr~atDi42+r(LrmwIzeLFQWG z%y{=N7SU$<@MD2Ir*|{uyqn!)c{dk*8?(Keu=in4YZttmtbecR-c8G?BKHhKuPKIJ zQ!?Gv@AROpw2E~{?i`@1 z1=PO-H0%Q!_XFVrK=XS*Npx_D6yWz?rt}NopqA9krh&Pw!&7})|b7`K0JPCOca@GT9J+KMc zgls}KA)Am*$R;d!6Xx)4EKbG2H+C-6r*^>$!Krv5*lw5$!9?$-l#_18HRT85&4@?w zqo0K2HrFkM!ae3df3xd`-s+YEc2o1;>}))cgzt&H=#- zKP`jaaL+ysz4R3AECS((` z3E6~fLN+0rkWI)YWD~Lp*@SGuDb|EJv>Pt8zi}4YQ=4!q{xvDmId~!9LNHOg;e4C? zX8V+?-UmncHY5JIXkzr6Vr9!ixf$sKjreY+|FcXE)-}nGGWo;;4A?Q$8l^dntwk#} zw${9rn_G?dyR_Sl?KEtiE{n&a9k3WrtXk7tykhlOH5;}zY^}x%w!gn%yYzxhD>7cN zYsYUge!{M~jGwS;y75AhwyQA8O}zFIPwy!xH*Y7uGF`c;Je^y)={k05=9tP&iJ{y~ z7v;tY<;JS@&c9Nx8Qr&i=S3h{5A^K>QVqbsE+E|qWSW3%7#MmT@V)`~ z-UR$_0afn+HT!_Tejs=NsC^Fzy${qM1R6d78V>>CkAUWnfygI7^a#-QDbW5I5IY8R zeh$P>0Nq~zi7$bkuYu$@K;O4;Zc-=d+;G$l6JP>NfC(@GCcp%k025#WOn?b60Vco% zm;e(fd;)ZCxT=V&inyw1imfW*9i@3k>4M);nkOMoLY{;i@W25NY(h36n~+V&CS((` z2@BqYIh-5iR9vLbMfydwRd^vd6)%Jv=Rz>ixtRl(L{}Z?@UR__08qk7m1= zrQzt~^llc6rT?r{*6V*PmR~~eCa+O9x515ZV4})ocsB(ab(1{xh-WbJsMhgZ=Xuk; zn`@1_sYph(QN5d{fc;9x#_F+at!ph;?i^jb!I42?830m)z(5*E{|ICE8qPjSMj_}8RLXXS-}3&BM3#`zl-X~ruN6+=uKwr;L| zQaWKC&O&9$W8lp-U7AucWb701D3xDJ_a-m*=Av^mMckW5AOcw+0^MR#{mo9sF zsn&lo_spBRU8_f*c~f0I!o9J399muHxZE2Y89bJ^fov2QY6ZLp0N;Cn|9zn9AW-uG z5I6(`KLTn$211_z^+$k)Pl3kIfbcP(`Eww00*HPAw0#M*e+|UG0Xn}0;wORb(?H@o zpyzua`2*1R6OcLs4EzkF&jFbWK=v14=!w7L`>h6izQ2}P{FQ$#_^JYa0Vco%m;e)C z0!)AjFaajO1egF5U;<2l2`~XBzy#>taCH$^7jbpb6kA=ydrR})(gnY_G*3dFgggm3 z>Vcyk*o16CHX)mkO~@u>6BfJ)bGSD)r{ZFLF4nixTH%G@RJ;&woD0E3_hudk-Q+#w zXC8+9tW=hZ4-C?}TEDq`&mvP*9DH0J`b}Pf&0-N|9Hu-L$nT+hGj-jY$Kcjjp_&w7 zYs2<;hHK-zZ`+?x|Sc1@al)0#Orx_eW%afEv#(}yZHTE{c@21f>u zWgXyu3aEM-sM!DnHUhzCfZAt)&=#P6E70&9(6}84KMyp&07P~G(U*X>mx1ZeL#9YkU0Qk-vfr;2fQ)B_aWf#1gg4#nm7tL^DdI2HezwCOCo5O5)w z=-w2=C(+f>dvw)$uda69r>pAwb#?XuUG+YwtB!JAH7(KU=Unu2oT_X6CG}?5!MXSZ zlW8vZZmxcQohA-O*W{tz{2HOAtPuLl^$0sZ#lDnAXv{;a$_! zo7QWLy%{{eRdfH|7@MQX*qmG4_CGFL_NRZ$%W2&*LcMW$+*8wWVJ-v{)tfT0Np#h+38>o!^gDIcbqn^)z>ZnEs+_CS&N0lSIWUt}Dyy!nBfOgt z|6F9YhO;0q@5Y4bGfuHu-1$G|$(v31+;RO9sy9V$C&U`NcKjy8tg&k@Y;D+~8!r@T zyT?**M9HyW_{$1oJ55_rVs$CwKRfnYON2!Rox^+W!|%a+-9;rfh%WA#hVybOo|fF~ z3hj8z)AR$>o98baGhMxDIaTDIVSHM#eA;wZzte-l(kj*+xpRE#joUL_Yu#oW&(s?n z88nucfQFZW##eywt3dN^Ao3az-2=4k1=`;LVs8SSZvpXlfbM-jVn5Jx07$+E^t}(H z4gv!o0O>ZkOCS{0pS6l zc@XMNBu({(Q*W366JP>NfC(@GCcp%k025#WOn?b60Vco%m_XqZpn5~~hUyK~8>%6S4`}gls}KA)Am*$R=bHvI(bH6XsBFxY(Z73H2*Q>%^&eA>2?Gf{E%) zDJR{GYswGYn-P!VNl|g~n0Ye}XC@!JW}$dB(?@wMl2<9-U>45Qnt9VSmUwefxG`J2 z3IFhzr*;2-X&v5ge=uFV8BFCCZ(0xSnK7z(Q?YNv%o_#qrg!aC;*DFYcyGiY90$e* zHKi<&Sbh+RNfC(@GCcp#=o&d!g zE-m8HA}%eOVoQs7YiZtEy5P5#=1IttkS8H0J#f+kn~+V&CS((`3E6~f!h$zp4)MnB zR5#Re^BZPl1yvBLAiOr6icXaE*IfuEiZ>i`liz@!Qq}w52;*kNKNn4mfK#kg9@5Qd zg3KuHOr9^z5yeK0c;Jgow0AQn*9@Lw5rQ0({3ufnjE8Y}l;$+H7Om9STD3cj?bK3l zc8#Up{C(!l#jRO0Z`$_%m#6FL$F+_x53l?0?%Rw3H8*2G&0W*nmitO93$*I{N9>(n zVp%AA=NRvuCGWk{ebe2uA6-6se)&Vo%galg4=i20blJm8waU}E)tjzkr)G|+-jo=> zf4U6Iak`Yps#T9LZ`>ZY*5w6S4`}gls}KA)Am* zIK`SUhk3(=?Z0wiJDmWhVssSOY`hSr=7nIQc{2w_iLN@zb=9;)SDRPps^W3jIIAtX z>RJ!9Yy#@G0sT(wx&?b?V8<+7RnFDb;2(5#^d4Qc-mCuxM(32_*e};l7KnQ?astlf zd(4t?^t|kwk!sB4U)9{_D~toz<=9Q0voq!|Ec@s5Zf-OD9G7;xv7Lsk(`E5kv;!97 ziB)Twi&w00LTuRDu(cX5*#7>4?a~W2t;l%6t{uP0_zAn_GJe9Y>Bb91T7gF1#GiiL z)B8ERn|E7ZFx|Vk*2tTRWK|)!GXbH*#IOq z0zJj;c0NP_f>_ecl6Nq;K-Ekms80hH+l1G8Q1iYKnae6n3 zi#nJ96JP>NfC(@GCcp%k025#WOn?b60Vco%m_Wf3pm)QSMO<0Ll|@r*Wf5;I%^OP> z{KnEe33(FoB;=e2&Us)HvI*IQY(h36n~+Ud@FvXR-8eWR#;NEISf-{s@atxM&;;RM zleK1|cjNpGdo<&9h>GDQjUup=PMC(XP}%aBdNY~sQMA8$pC?~Krjo`o(DohKz+sEAlZ0<#??UB2Q*g# z5kC-J542SQ?bSf62I$-b!~;P0W*`v+dbR<{TA;5ENQHob7lCv=klFdSGE27MZv|gc zz%RfAm;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28#T$w@6mKZrP`u&I&Dn%(LN+0r zkWI)YWD~Lp*@SFDHX)mkO*qAxFo$?U=Y-A)<*OpfS5CzX;fA^pOcZbCamY>H1AgXV zz|Tr$yZAt@j3VLSrA4NCIQY2y-5bbxE0uk69`odbIQGWn@YpVCICf2j*El>y+Dv2X z&}JK3=cREslRWn3F?_hy;euu5ajP5#z(5K}p8_%iKz0xq zN(0`X0N)wtH~ycg-*D~?6JP>NfC(@GCcp%k025#WOn?b60Vco%m;e(fd;-*OxWI@D zjJUvPiY+kW?WK8p>4M*0nkOMoLY{=2_P}WmY(h36n~+V&CS((`2@BqYIrJMVM~gTW z?F&Wwf)@fWgd1sMl8O3FF_aQr9lb|at@r9`=Y6`WzF${oAJA3rgSzS{*HzOJor=yy zMaQYS)?0=JoXekJGR=g_IHk&xhkcX7xLGK6jAMcxi{v*`y_s_AP1tHEH4a0u$)(<0 zRBp^xZ@k&Rd8#{BY8~IKI%&Fk(|V1`H-qQ5YVO|~29O2==&oDc_CGFL_NNl(14|b# zUH0%&t-5-IdgJi8wASs`@l3wKkwIfQ5BP_Gs*07+Y`j2VH4yXxwUt1~57e&*8mfTC zY9L$#G;abT0U){=XbS@E+kjXt&{+q>LqPY7K%yS#*$E^YfWBQosu37y0@7h1^E!}i z28LPy?^}TH9l*a2sM-(I8~_6Ett_(y-(Sg_iZTHvzyz286JP>NfC(@GCcp%k025#W zOn?b60VYuJ1gPFny`g$T^@i#V?{dy2WD~Lp*@SFDHX)mkO~@u>6S4`}glxho)`U6K z8(Jr{PN-kmX`S3i)(QWb_-mS|-js<=qN|ooK;1T=->IvvTd-#acFfXMYA)?&&&9I5H$ZQQ~L0;a?*nZ6%F=bpPK8oeHQN5XR>do%4)Em>L-rQ+8FSp6m zn_FBp9Vey0bGrB$pua_9Km8<)qe)qOQC_XdXs zk0k~KKLlz!flwDv9|sx^1C8B4_$bhv03yeMXb;fV3$!PJ*jGSj9}w>cx>G>n6wosO zBnN@MG?4lc7{~zWvp^;bWX}UbLxA^*3JXM=RYL!h!m2gGnzh2fI$`iBVeQkx&<0`s z#tKWBrQw+hOTiZy@Cz^jCcp%k025#WOn?b60Vco%m;e)C0!)AjFaah&4TlSjxX_3T zji%T_Bi?A5H<~W^jiz}L@+9O*$hi-k`@kk-6S4`}gls}KA)Bz^O_)Q&u{aei3|bgA zS{S?#oQfC1jdLNGsNs}y>dm+&{=mZ-@hFx(n16H4hbdZ49`kR;C)ljGZiPmlbOQ&@ zPnmq^*v65$?rcv2|#(jjdC=-PpP= zZKq-DbXhzWt=8HLRH1#ftH`LJ)8~O<#$H^YXEv=@!mlDz}BxdT7s#Q7xR-Eh8r2 zxINRgikEHUnSdk529ss8upuaH+$Idy3Y+VMk&rO@qOh%A*uGO3YY=ws62=>a-A%$o zSlIKrFxf2ZYZ0a*!hyGi>8LQ%D$KSChu#%>-xvB03jH4ls}2ckJ`x5#76v~N)*cat zJ{8t~CTut+Z2VjpJ|S%WLJXbAmo#)NipxKk025#WOn?b60Vco%m;e)C0!)AjFaajO z1eid<6QH3(Lx+YA4ILUfyeT@HkWI)YWD~Lp*@SFDHX)mkO~@u>6S4`XSQF+jbhsw} z##xh3!@{Zf*Q9gj;Dvw-!9+uclX3E!3RJ3k9~`0PjQHmwH6f->v9ji2=Umsc8bh_2 zzEX|ju|nQ_T_5@@ete49IsXbH#sP=sdhMLl1lT#d3S;MlA?A2m!%x7@@xLE7-Oi~z zo!idoI(BO2`0N}7JEu2#)r_3!TG#dw%8tY1(3(aQwj4*Ll*pn3|&kOs8gsF-rzyYr?y;_*@3A2^LA-~Z3l+gFI(7!=gwNY5} zj4<%5Ft|lnyHyx^PFTNP*zmlt@daUchp_o2VdQ0D^c7*-tHSo(Pr%NJz4ip>=g`h! z0!)AjFaajO1egF5U;<2l2`~XBzyz286JP>`Pk?p~*B^2H5!WA0vGqs1O*L;*UGUpf z^CaX+$diy$BRDmJO~@u>6S4`}gls}KVZoa)hn=IGip%-AoWF=(3oitx;)QVITnHxG zIdfp0=&GY!S4~TFwRwfEDjtWiv)ZDouJu66CZKK`(C@^qTd-#acFfXM72{y|qq z@6lE3z4~up4o?}5eWkJvKNiUIk>hkO-(!}rqvvJlj6<#&)5Iwj`$moVm?Q5oQ_?Ax z-!}Gv%#rsLeu~ai_H@RaqVotOBa6qSUEEI7R+Ly>%J|QY{nip;kwNG1Ui8kL;(>WQAneORaYl==qGOCU4>4fd>5@X!YbQ$;KbSd~%UAJ60 zL}$9VJBP2O>Byl$XL(&1YZi942;&i9_uIlmRM^uhOtuO8-W8_Wg##VJbWE7}P?+r$ z4s{8=p9pcwVQPXk$hP9jVDo4f8n1&sltDlrkn6sJkj|Kb&WcNd&c&@+ zf=(Q6j;FU%1f36Slcozg{U>t|(y80Edh|g$75hdAI#!QeYx;l3Bk0JXL1gg?6RU+i zK4G#_*yk6f)(Z!!gz0Kwrbd|EBpeC|y<3F7twR5E!m91Un&*Xq7lgqb!rGUFp_hgA zuLv7n6*le`hF=pl?-54!3Zrib+ujtmza@;lBkbHKjPDnA9}p(q6ZX6>Odb^Wec*+l zlRD((9G$}7BelY!3j05P0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3_zCqO}mOOUt( ziA#{C*b*e(u9~;2F8J-Lc@pv@)bg{4$Xe?;gl!TbX*RPO}oR;Z|vG!V_T%nG`0?Hwy||;1<`aK!-rcv4y_1V8@9j0 z*5YwqnxEs+%++*~FmybF+g3r-dDWktG)<>9b8vJ`r)#hMN+rh*O$YPAuG~5LARRe2 z_$*z*R9rZ4SeWh>W{wK83E|Ljq4$K)_l3~^rLgL2Va+$fz_-HSNn!12Vdy(y{rAF# zAB2rR3BzZE%|8nx=Y-J^MR#{mo9sFsn&XpQ96U?w`%U+8$Ykf&)@2{|8d!} zKmFqe2SZmmIu5Pwn{hcha%^x}YK7T4;ZR8E-68b7B=o;5ta?RQ^QthgTNr#zSi46U z+AFMoL)h@9u<3BYIx zUG+P4)pZN@%)pLWx~iP3Q{yodra4fURw|pWY?s?PS3h55wy(1wKS4(qW5zLQj}`J+ z9HNug5S_^sbaod;(DA~}@l?OL8iLMuU%YI(pwn`y$UVdGaEjsKluUQ^J3Z(ttzzAg zJI5#JI6N+`VwZJ1Lv-ZWAhMhm`o9xaeJ`x}K^XW+7(64a{aF|~C#=69Z1_dk_{3^3 zyh_;oq%g8Z7+ouDTPJLPN*H@u*ttO%-ze;UMwob3*t12L+$!vQPMF#*9C%)senFVo zAzp(y*u;D#nVn^_nkOMoLY{=29l_ZVY(h36n~+V&CS((` z2@BqYIV2ssQ*l*4SM^(|x9~!6DqaXT&V^v2q{Fc}`Hc)JRlN_6uyaQIbJ4`eI>pM8 zhoCc>i?c{f8n?NRh4Kq&=S(>}2h(93@M|2JSU6T}i?Fp}`#WR5^OBWgww)7#nB!?W zB6iMeZ+>RFol|)_x1G~NfC(@GCcp%k025#WOn?b60Vco%3ZDS&9NIawb7<$#&f)FS*@SFDHX)mkO~@u> z6S4`}gls}KA)Am*IK`SUhn>Uq_&3gaeA*RG#lI$fI~y+qTnHxGIdfp0=&GY!S4~TF zwRwfEDjtWiv)ZDouJu66CZKK`(C@^qTd-#acFfXM72{y|qq@6lE3z4~up6i*qB zeWkJtKNiUIkrQ<;-(!}rqvvJlj6=YgXOdjwoODA6PEeVA=*W3N6TPQQIpDWAJW6vK zTZ{Iu#@4FcVQg)ewsvi<@$VvSrm=NsvyH7&yWQBjE^Vh_>x4&R(NY%UiB)TopR{_c znhjeUwpQZ>+uvWXU3$R=Bb91T7gFBgyH9ST3_%%(y7_; zz3Gz9wMOVvB%|7>l1@`##0VV)NvHP*%axK&&YcdAOB;OJ3Yi7R2ASnkVdOJm^q8>i zb7A`lVeAWG=a<6x*TU{^go$s3Jtu|9)55;*gsJa^13w7UKM6BugxQ~kL+6CvA)&9r z2l~Cjs@1|8pD<7<4Elw&>xH2zVSTl*p+?xaNf-_Yn>Pz1L1A>8u&q|uUgv|P6ASq` zLZ|QxFbaNuR(=5{zyz286JP>NfC(@GCcp%k025#WOn?b6fr2MMNr%gjxD1KQkfzu& zB;Ki-cd9PbCfg?bAw z1gGMKaN}GECQ3Ta->`HuUg4-164S7ubM=$b33GN9Dr+8NbgpNjjasS3{V0*moomdW52aTSY*&Syw^Vb9=z9G zRAO^g&UKf%txA>Lj;AFaR~zIUPuKTi==@N<_J6u>GhVzIFWxoHZMm<+vOuf8f5hJT zC6&PdyG!quGOQD(P`ac zcb6Eyf4Y28r%S~mnB($tmCjtdF3wm{F31B1r0Ll}EW*!i+B{)({s zRbgVcu;(>la*wcYuQ2t7aNtd0`YmDR9btB#aA?2K`>xQ}F7$T@t75{M4~2nFVX#YB z8yAKS3+uau4M&BI31RrSu(?MV=@mwk!nUu3?R~;nzpyhUjGq#A4~U+V7^I%V89GdW z2`~XBzyz286JP>NfC(@GCcp%k025#WOkfHmKs|?g4)q-BIn;A_zjQVsn~+V&CS((` z3E6~fLN+0rkWI)YWD`!YCd{Gda6$f!vml>(g;VjbN#U-D7XmH>6ZM>VoST#P%%FLg z8MIPaFFqhWUTx@HzGsoCwhlfn4?}0%icL-{XMuP(mkx9ECyV86=6q=WzsJyma!52e z%^I_IUcaGpn_>94v|EksB+bxy3?F9oI52fb#_~9{-(hR2xoX!eH++1$wT>bnyeVtOJ z@(^_LvTAaE|50|9%Awh>^(d8>=;&a&j!nD6aBu9|Tw`0L%`~YLn^cgsu2c z3!I!su+8P@$bR#u=p>=%cm|)Ugrl>!zr}P%r}Y|BbOz6F)!e@~CgNx^h38hc{g2C* z{pla$cPjSTuN;qKM>JhKEcr`1a%7NM_6t)7gahvh)9(v22Zh-WghPjf-Y%gpF7zK3 zR&@(&jtT<_Veq)HwnrH171k$(4POZx`-I_sVRK3tIVFq^2-^mQ?P+1`M`3417(XlQ z&I%Lfg*`*UWJM*|=M|<_3kQ6{bfqxkuY{zNU0+E_r|=6fDlDq7|Kk^60!)AjFaajO z1egF5U;<2l2`~XBzyz286DW8BlyoTRP|~5KLrI6XOlK3a3E6~fLN+0rkWI)YWD~Lp z*@SFDHsKU&!W@zgm*w9$%kn8%I2Hez)b8xO5O5)wDCv}mccQD7O+ei?px>#hu3NBY z26oKSRpnfr9*^NL&4I%NAw?f+$jls|5Z4=hi3Ilb*U`SZ|qA*l1 ztlueYXb?8;5{4Uv%}v5cSQvd>*w!p;ZxO~K!p^sa@u;x7RhVcK_Pi@hwhQ|@gsGTt z;6q`$Q<&)zX5+%4!$R+;Lf>bi=lG9N&#_R?VFFBm2`~XBzyz286JP>NfC(@GCcp%k z025#Wg-?Kb4i_MC0TLG=O|bto)N@KXH)mWkgRVAGiYG z&`ju)GWt1}#^#JYK4&uhoTkF~IbNtap6Z6R@N>@W>o?uc8BFE&b6OAWnK7!LGZ-2% zKF8&8X{{~R{Kx0Wk-=l>5&Cy~3gfGV-9BNWQrP1cCf5u5s)VU(;XsWry-An}2(z1oLqVbU zIiYX6(Eq%!>IGrV4q@P>weWL-FR$hJoWifYDEPfu`30B&6JP>NfC(@GCcp%k025#W zOn?b60Vco%3Z4M{9Qrx*bLi*L&*81o*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IK`SU zho8e0`8UpreEJnm#lI$vI|nZWTnHxmIh>c1->jfg)%)PpZqDU<7EO$&Q><)xxH+zs zYEmC%@^PG?(|#Qv)r>Q+HO}oTIN){9o?dj6pXyW6;fA)7+N(N-PVs>ib9R zonK;ED0}A^@0}&@z0-Zu-LoHEK6`%oL(9v{OPmiZUA%PJ!%MZw)4AQ8u4AWWj@Qj` zYjqdK;pWJZ!C~1Y3^odDn}ngTu>N&nL$k24MHr3NfC(@GCcp%k025#WOn?b60Vco%ra%I8 zbGZ75tB<()Xo{^q;w`Fqi|T^kqM9cmPePuAoEX805o|&>A)Am*$R=bHvIz^`ggM+C z<+^HGqN~j-bXD;<%Pn=W@2_FX&$?brLqco?nwP^opY^|J~qg2W5&}vFNt`O85 zPgA=ee$JcEZZ_S|xz_BQieyw9)z4`PjF_F{_Dt6*+HCpH&XFU7$C4E`oEJ6@3BwhB zu-Pk&tQJOn!nR6byI&YvFYK%m#;b+hHNwOuVNXDq+$`)13RByJ1GU0*oiGy;W?vKz z)eF5Z3w^H${jUnEb_;7>69)DOgL{RwZwNzg3hUnzHoPNj+$Rk07d9X8!_SGl=jZGk z&dy;1On?b60Vco%m;e)C0!)AjFaajO1egF5U;>3tfPN1B9Qrx*bLi*rKIv>iHX)mk zO~@u>6S4`}gls}KA)Am*$R?a(O_;;a;fnklXGK2!3a8>6a5_LZ`ijP zuWM8ce`#3Gx%x@zgy}j9l_ih)IoA|&N=2lZJ`s;n`L$GZrrZdfrm;roJOaPS;&Ev! zjO{dSMTym=jQ{M|Z!HlP8FUWswGY1s?{ycI*x=>3OO44lmlT~a>>N+)rgczsf~%TL zS9JPM<{qI_w`=w2BXp{(M<_Z9icarNi@U`5_R~c$a=Mhqs#VlC(k31e|#=V4*ITiAV6m`DhFjti4L!oFT%Dk&WJN|^2w zX8MKMlyK;j(EFXx_r1{ngRtr+Va*v~;Adg*oUrzSF!YPC{)u&9!zy9plfv*CVe?vH zWSubjl(6k-Vf%)4aCBlD*U`}_{Q8Ruiz@8@_yw2%6JP>NfC(@GCcp%k025#WOn?b6 z0Vco%3Z4KR9j-#+DkQE#nqsSvc&lpOs=DB}s^&?^laMDNCr5B{1e=ge$R=bHvI*IQ zY{G&!VGc*f=2TqN&qe)q+AX{goQfC1jdLNG=;+Mj+?>2;2F=6Fpq0vU@qxKqS8F<# z?^$H3vV)JyL(|Di#93^fShFB+L1(eN(`w3Nf&89--Z4728RK`RX}7zb5MeA@%3?gR zYE5(TiWQ=W4O<(wR^tWR-(Rp@dcmd@886ti<2M;UVb^5DkIiG(bmN60ZP?Ly%y@$X zKOt{$XureO;&EPbbIf*h+Mwrnx{~7PoEvz5(j1-E%)!weow|+ot96{N|8eC$Nxu62 zYbZK$WKdZG!q{eEXHXd5ChV>iChCMeAz|`GVPCy4wNp6IAWZKPW*UXrCgD(6=-n&y zy&?3!DXe-+So4lBuumA=FRVQv4813;e_z;eP}ul^FnmbZ{E;y7u`v3Hu`Pk@RJ z6&)%%RCK85@RsRpLN+0rkWI)YWD~Lp*@SFDHX)mkO~@vkVojJs(cz-}8)s2I6$_{0 zUz5_Eg%<)Y1QQjVVrVD2I(m<;TJP1>&iiy#eZQ{GKA@}K2X)m^uB)acIyIh)8jn+T zt-oaG468a9pJ33<<$BK5&#zP1DOJ`y44q#y(5B2hcg7R}XUtzx_RnePTr_iB+GH3y z;jyOZT%4X`wxJV;oa5rsxcw->SKPZw$ZDF#P7OTiy0QE?f4e z66XU;7cX7*@KUXH%Lqfq3Y)2E$v6xhIWm|mUkMX^!k&I%G9~OgB}@$n2L^@dv@r9d zFq;t$ofUe|34IrY{$GSuPoPmOHLHYyCxyW^!rHaM&^lrLQ^JO)g^e48;f=!PXM~Yw zh0!g-wynbU=Y+BC!p`S~@fU>MJA{dsggq|{ldlN-UR@7EC$)P$hv*c3^+mz&(8@2s z1egF5U;<2l2`~XBzyz286JP>NfC(@GCQ$GMXy|Ya64xMc4bl`_gT#AP^Ip{jzgIO+ zLY{;?2{}4~qa)aaY(h36n~+V&CS(&9ya{s{Iz>*!CH-8|@1Wbl3js4Y=(gN27lMg~ zPMJt2x@y@3)NKR$ox19}1$$;-$1Gh{&ebXL7y{EA2u$!tbTLG7dpZ~QFEZQK!PJ;M z^c)i=)Sx0(DtrEqdGclxK6hNdgoe(TGjw*3W#~L&%+fL2&`Cnh@eB@$q4Q4r8Pg4& zmQzLU8HR{c3=yYfx~t#mL1k$b>yF$xrlE7I@q_;&Kj?DVF;1vDY#E25BZmf+B`i$6 zE*xkUrdx!Wh%o!Ma40JD?iczF2>tH~tKJvZ925pV5C#tkYd;c(J{H!0B5XJ!Z2VLh z{!G|>Oc?oG7(F3u`$E|Mr7-rju=5*X{99r7Nnzr&u;)8r@_S+555m+>!htiw^v}Y~ zIZ<@77pUlPhz=8A0!)AjFaajO1egF5U;<2l2`~XBzyz286PN-CP|=~HLq&&*4iz2V zFr7`vCS((`3E6~fLN+0rkWI)YWD~Lp*@RQ933DhqT$E2cg_@S4o#Iry5N@ao!9+!; zl!J4|H8kjIo1}PBl$|^ropHD|`B*p$#lLZk@>nFVQqXZZJT~nPI4=&2W|?bji?o@> z)}hTdwodJKV>>w`behHzbS7qmP8x2Gr{d645Oiw8Pya9XZH5$cvy8Qw=C<5dVp*V7 z-#=pS{1VGT**nL0?<{%mo$j0Np8e?Z+4IXET3%lMk4ZR#soa82>!Cd}Miq1__SxN6 z2s#drOB>9LN6?W&gUC|x6qxl2hgJ){YlObFLjO8p)l?K8sAv%>l< z!iKHF#^;3L?ZW2gg^?G8(H+9JmxS#v3uCVcJ6{#XcMH2;6DIZud-e*GZwUL|6sF!1 z4!k2w?-OSB3$q7=L+=T_9YSC1DF{0L51--;9nR2U0!)AjFaajO1egF5U;<2l2`~XB zzyz286JP>`Pk@3BmmqNo5|6S4`} zgls}KVZoa)hoED3svBxq`VBJzgUSe%5nh{4MR&^j>n;Qn1sx8}$!}m#sp@@jbRA{U z#E3e@O64Kvj3&~I;@IT*(i~B7Tq7REvWI@ol=E|TjpgS|%m5uP)ErOs)++cpzdZMn z>3&Y->D+!!*RfMG$LHs`JZ`P)E$cW2=*XeLWBEwv|5#Y{iLmB~Fz~4`_?fWwm@xFY zu>ORw;R|8om%{MZ!sc&;k#B|3lft&s!uIckvG0YQKM3PL3A@h-6F&=k&IywjgnhpV zQ%_WZ1FMATCxw|c!t7e%&^n>_|7Y+0W1Fh~_}_PbY^M}H+EtOG`}l#F5dOJW?)BcU z$#N5un`8X^g~|QH@Rxs(d=`_h5pGDhYCbW{iggw&vp|IcWfrVhp^TzqRxP#agd%fR zm{@Smiena-vueeP^S+-4oG{APLAUm^&oh+ADW@&ndG4I&U+>3xxj|^`OTK zdN;bIpX2kn>E~2_?M0J8RQ3;l04Bf$m;e)C0!)AjFaajO1egF5U;<2l2{3_*CqO@k zeh&Q{`Z@G-c(ZgiA)Am*$R=bHvI*IQY(h36n~+V&CS(&`O-Vl#KA^Qv^}PYjVm@d- zBX`WzMDqeo4Bf4Xr?9{WlmMjngsd@MtyT*{7ClA|+Ppc&=Ul<}V{ zpypULVIBMMC*#!y1U_{~dhxp&ghd~KBUDA-8q_=Wt?|;}91hU2+bkaG=Ga=lgP_y* z{DHCyI@cPY)0By+V+uO0yG9Mru|m*snkFFV;LxPVvK91g1N|?9f$d=M6)^NF7=8_m zybiYS2BUkx*j_NP5A1vsOzsD}-U3r^gFWwp=>uTzdtl~$u>V6a`w=*J2+SP@^GCqK zC*bg>pyMd$>;YZJL3bMToCLkSpsx?~XTZQIFxU@4Cp17o$3Q`c2`~XBzyz286JP>N zfC(@GCcp%k025#WOn?bgJ^>0kT!O?UNL+$cWlNBFpK9Kxy5je#=1IttkS8I>MsREd zn~+V&CS((`3E6~f!iqOx5kaSx6K51b$-q#ZlEL+j4xuu_-zICVjDn8xUo6~AR5q%U z0@H|{bM=$@N%M6U3sVV0bfhjbrH=**Iuvw-u+O$MFOY&xK)N}$w*70Rp!43(J!Kbk z2F?_ppwqf*)z}ksh5}a!IVEY?urJxgSU&{$Pm0x^O@!PZV2VeqBfC(@G zCcp%k025#WOn?b60Vco%m;e)~cmfo3DCkhop`b%Shj&Y76S4`}gls}KA)Am*$R=bH zvI*IQY(h3+l{H}zL5FS%-4se%HI%dz!3$wpT?l0qbQW@IPRSF47Rrf1D~0LeL)OaG zj?U$KmXy`kkzZH3qoY8bnKdNfC(@GCcp%k z025#WOn?b60Vco%DxUx~9WF%TLL@Fks)JgS36UXn>MEpKY?0i5I zEe~qq{6m`Pdsq_(8#U4Pm`0W7qROKPxrl#6)VchLoOE-!qI3208L%RnfNw^-g_jEP0a7t@`^d>TG>GLvG>O z7}Tsme_~YI7RXnOHlu2iTa(-x^%qQkzF@laf=LQD<0v|jKNTIQI7X@Jj8=3msyXFW zbQ03ev2}k2Mdz6#&i}=FtFFM@Bo&yuW>^gm*bR%+mIp`ez29zFjJ@;q_s+$8@379i zXWo;`=iT4<$nwTUyYkS|N0%;pe5o40#w4Ag3!7EzKlIP5`1~za^FNm@``2F|Y-zbl z(J`tm|Kmy<$1D}6)-BgkbZ}@=WqA%vJ`Z-S22(G9J!`@Ai(v0MFtY*de+kTP0tYvP zxvgM+8(4T59NrE(f}nFJ=n8@EU7#lndfPx>1oXcF2HL@32N;Ti;dj7D3~Y~s(F7R# z08Dg(od>~W672dIOm%@h-B5JWDJnXK$}hjD_-$JG126$5zyz286JP>NfC(@GCcp%k z025#WOn?bgJOL^?RCK85P|=~H!@H)l3E6~fLN+0rkWI)YWD~Lp*@SFDHX)m^%9^l< zqC-1{b_z8uGwqb=WT)`ADSw+~RCF3(ooJ$Cy@Xt$u89Ff6Wuqj;E&~X-Y4uY;B(47N4KY`vn z=sOSk3t->^7#s#eO)pA@9bjY?*zN?Q&0x$0Cf0zRZZO#bc6q?mda%a}rZ>7J$diyKA!kQ$b_AP{O~@u>6S4`}glxi!H(?P)$Djxr8ZtJCwIJ^gRL_p^opsO8pcYvNK=zRzD#Xx@?3?#td z2VkfZ3?Bp|NwEE6Fxmyiy1_&W>^ur4d%&*aU@8svoCMRoU~eCo$$WkN^!G8agy|Xz0+; z;SJN-gls}KA)Am*$R=bHvI*IQY(h36n~+UdWldPb(BYc=>9Z!EhJ_;d+oW?>%L_qX z2xT;MI5nrVi9z+E@8MBu&ZvJbnH*C`4yq}^&M}W6wT%0cuG-YD6cc!?z?*OAORwYO z|DkryY-_Eh)@D&JZfD4?VlmhZYSy4XF{*6~yNnpsLIwK@ix`GO?Aa@Q_Yi*Cm~NlPL1Hy2sR;`kWI)YWD~Lp*@P8u!XkE# zPz0Crb2)zvy%t^wir|GXeJ+GD+Bx&3b)t!bjhbkCOcNVdXrk#UY3!^rXrgj?$T-uCN;|LjP-)`;B(6 zMyE2e*F5qb`Ce;{-DGKAV6C?rg|N?-Pwh5~L;5+kmNo07r1RW`Z_6&}Tx*0*QzoX4 zE$Ku?jnJ{!tZK`_2}wFQHYu~jL01BFe*k(qLGMA(mjwMEgMltE*bRnKVE8B)=>gl1 zgV8h?I|(Lw!OlJ~nE|^_fvJA5X8=rR!QQiAW)SQj0<$@A@Fy^r2lMB_LIE7U06P8v zolmcmbUg#Qp9MY7f!^oWNlC}IdL1PlN;*t{2`~XBzyz286JP>NfC(@GCcp%k025#W zOrY`!P|~5KLrI5{4kaDlE}c!tCS((`3E6~fLN+0rkWI)YWD~Lp*@RWrgheDBF3X=j z%kn8%D1yIDYIjCn2=YQGqokuq@kClUb2ZVtKodiEYvTC5nuy=0iJcE+V2u$q)d%|G44eX78nK!`x zb}-uk4o1P;J77Kr7UJM=0(5)`IzIwkhd}pX&~pUzeggVF1^u6afn#9sa~L|IFKFl( zD!=-o;`eCf55NSN025#WOn?b60Vco%m;e)C0!)AjFaah|@dRk-a19dIAaM;+m90VI z-Ku%F>Wbg3nkOMoLY{;i9>L)eY(h36n~+V&CS((`2`k=&MGPI2BDkcVOZv@pTX-QT zf)~Q{xe&@|=q%*uoRY@|EtCe+N+B*jWT%WWbVmJiNm+#*`E?~oIz>F1tIe7l-?I>I z&a8_M1yAVdRIjJ=Z#gl?XscCg&$!RJ-K||{f&<-Z)~(H4=sIk z>9WU{s@L*#;`u{kdpg~3j`DP7Nl&Nm=!85S9Gmo6PJ*FcFx&@5GGO~DFxn5s2EarX z>^uu52f?l(FqH#)ege~Zu=hNeDS-VK!0a$M*tA|U=K%Aoz=9JTZU!BzLFWsgYc1%0 z5%jDBy&FK^OQ3%f7}yL3w}PQOn?b60Vco%m;e)C z0!)AjFaajO1egF5U;<2_@(Iw>p{GMnhn@~S9o{vaO~@u>6S4`}gls}KA)Am*$R=bH zvI*IQRn~+>JRPpgpFS(|=~*a(zfGEV23`pALMWrBQ^)B!rB4o0ME4rQh^ce=6FKW< zl%_N4pEqdj)C*Gyn$9J4X2hYH>?aL$bky6V_EKvzUD}$}1^T~h)Z6uKtvXNND(Y-~ zYq_+YA-9UfU^A#$gZ{**w&9aTn^85%tx0Z;`U|E%Uoc&I!6c0veAKL-oT-1ptXlL> zm{m=Gp+=1u*!d$Ao#f9{ zbT~?f2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bgK>}2CxCn`hkhln`$`&E6S4`}gcWbXB8pCpBDkiXYx-*`x9~zx1TTc? zb0L&b(P@Bnq6xX8TobKZBnA{sbl)uZ%#l0hYNC08MvF(cc;<_p53Cd>O_(m1bFO~A zq};mBqSE{v4Z_T&K|fRQWCgyjn}uG!Q#$buhde zjO+p1_kz)VVC+pWu^;Sw3rxNZcD)Ox4uC!HZIFIW`uzNfC(@GCcp%k028Qq0`zm}=g`lgpF=-~w@YUevI*IQY(h36n~+V& zCS((`3E6~fLN;NQHDM7yhb!`@&x(Bd6^h_*lg6Fkg&;44GWt37QaaH@%Y&LY|Bxp7 z9@fOcMoqLmriqO!G|}{w6n0igYljlgC4or_I-?$SQ{wBCFhFOtno~EPCp2ox&vpEB zzA#Rinlti64RmzIo1xP-o}+`=ItDo(=He`!8R`nV(IWJ}X1U*J2WxaHBYVvw?~(7d z*4Rx_W3krjM$2p%EjCL^>N&PPk5@W6XT4vR-O(A!7I$>wNA}Gb)6r>ZF<+_P807$> zHs>fGXKYCa$0lW#PB48C>`j81kHP*fFxw3droh}$Fy8|fj)TK#(D5bc{0ek^4Z6Pp zJ>P=f)1dD=(0>LDoCAa3gP|Y5@Q+~RXR!SjF#0PP`wdL|4tD+lCZG07c0B{8o&|fJ z1Jlogy{o~@3t<0RuatDMFM28IP|{%nOn?b60Vco%m;e)C0!)AjFaajO1egF5U;>p- zfRYZEA#oWJmmyWzG9=!sn)j-%_`Rxm67nSENyyO=938rIkHZjFq9zZNLS{X zk0p3Xj=Cx5WAW6^&bcV){D;~(v!`%^&aUz7oQqaYx$T^c6mx7teXw(mIDRg>ozr}- zxSi8|;_U4cvvbUHT29O13E4R~Hfgb}2eV#qa3h%Wf%z?9!4D3%f{yK=^A*tbD(HR< z^t=vwcZ0q?pnoqI*arsR1Vj75@LOQyZLs}aFnR!ty$2@V2RlCmlOKUyhrrZfu;&Pv z{sip(6wG`E_8$YYpM!&6fVmT3{!7?7g|BGm7%IQ=qT=^wRU__Qk&!QUo*I}a|AbG-YDyn37JLchm@-S~)#vXsz+exjJKy(7~}unI!`jPJzSypyNBxc?NWy1Kr<) zo*zK(kD%{o(Eke<_!SKP28MnI!+(I0r(cq6e+G;`3&x%U6VHR4tHI<8VAonO^&;4_ z4oq(VdtU-Go523fV0J4wxDCv`4Cc3kg;&7gS3ySzbnbddN;O~@u>6S4`}glxi!H(?P;r&bYM)z4M^2I?)m5EQ`+VftJMWt4OjDV}Je zb*?6w7iePWZcQA&R}=C3G_msmO|(3yiSrL>6nS)!r%r5KF=FUk{iJ@_SsG+Fk7u6>~UP0+I+^u7i9-Uj{ef`J2I@I5f}J{bNGjC=&P z9|EI?!PpTn@d?=ZDVY2W>^cUfJ_mcg0MjSH-Y>z-S786wVD=ku@LMo<8q9wO7S4df z=Rn60=*)qxpFnpW^qdF11sFQM3p8{%Mu!P70Vco%m;e)C0!)AjFaajO1egF5U;<2l z2~G<0a_(9ofwLqmtROlK3a3E6~fLN+0rkWI)YWD~Lp*@SFDHer=DVG%=zYx1Yh zntU1-ir{aP&Rq>J1bHEp(a>2atrJZ&J*A0(RR&FTuaW3jFVVV1VnC6*ZkBs!@koPd zp){CQ3gg9x^d7I4bS~esq^!b@{JIh(oe5huMa7&&P;acIJRZf{u1B=Fkxx<5DP7X} zx3$({wAHFLa%+;?zss$`rl^x6>3F1{V{1LUQA#>PAHFkXl1@B-XlzNRDL6{fvD#*- zZJSIJ8KZ+^lQPR6pzrC8lKyAFz_VcRIWY7*7+wuVUI5$Ig3%Yj*g7z=0qlGUOl|_Z zHiN0HV9z!%{W93Q9n8D}_P+{dUjqkU2Xni@{2s8d7aZOPI@&>J2k44|?sq^>4D`l9 zUjp=h00uh2;K7Yj(g`Iua*R&pZ;{$$5S9JIAAkuk0Vco%m;e)C0!)AjFaajO1egF5 zU;<2_;t5dF;W8vHL*g=|DqDucyH)dU)fK;6HBUmGgggm3Jc7d`*o16CHX)mkO~@u> z6IQ$li%2>~MQ~L=SM>|(ExZsE!3$yfTnJ^9bm}-dr}VKw3M8fxP3Q6_a?s7?g3i^? zZ&2B(7or45rzEka6#3@z_tZngnRTs4Jzk=wW2v>7)Z3&1Q)@G;3-oP`db_@@Rp;qj zMV+m0Etj@4%40Uoidog6Yx=CMn(Eqh|HwO#KsP z)uMmGtZMoTHEP5tcNnCW^Mw9}t2G_$H`pwfbe$RM3cJxF^uK1g-)IMGbSfiz%_HxT z@3q$0O_t^b)_SW^2>WdL)NZo`q@iPL+q6k)I-l-6T6Rq*evMf=Ll-uy)_>?fh$?@O zyKb?X|G8}0zwF9GOCMdj?D3^)(}7W%j*yy8e9Kjujz#VJ(Qu`^W04`nJB(6ik;f)= zmLp*36EOTK82JorKL$oW2V-A=i4$PwmtgWMur!0h+n z;16K#M=<|0Soj4T{uOjw0G-32t7(&@+W~r3fnF!*YX5Y(h36n~+V&CS((`3E6~fLN+0rkWE-+O;|+J;lljsvoN2Ug(CRdq={qcnVdN~5$WdG z;-?|#M7EwSyP(r?w#GU~_i*YU1kJJxC^k9nOKnIT_^2nsdvKfqS1!LR5#LHmkb};!0*!3!ydJXJ(9Zc^Ad-s5uyE#`D!=ri;Wbg2nkOMoLY{=29l_ZVY(h36n~+V&CS((`2`k=&Mf4nl zBDkKP>-kL-TX-QTf)~Q{xe&^z=hRE*L=!C!YU2Dun&^9269*eL(e{`oHm=Y_(^Hxl zSY^=Y@cilUjMyn7PwJ*b*C}C!&R^!t$j>&ZPa1IG6;Nd=@})QE=iH)yerg8jv{lB> zNk}!v*8QeW`Z?>KU$xMBtM0_ygrPPwtcC~dhDB=2gQND|Z#OK)-ue1_=iWb8v>T#r>T4k$rQ<^mB#+W~*KQ|7XF^QS>oI@;h(Q_x%m~ES5{7 za*FPhB8xmUDYASACeMIf=fKqWV9yU=`bV($XE5^%*#9e-{S6%a9nAd!=AZUS7M=lz zp9LMwpwk7q)_`s|=xG7H9?-WQ^n1a;Mlk3DLtDVGAB?nu?Ex^l1B?a1#7?j?1SWTZ zU12cQ2KGdJQqW1i;p6-q3OY=H2`~XBzyz286JP>NfC(@GCcp%k025#WOrY`!P|%^E zLqUgv4h0?FE1gZqCS((`3E6~fLN+0rkWI)YWD~Lp*@RWrghd1$F3F!hOY$jLD1yID zDtEQK5afkWMnOjkIEwUdZq`J{98I*&)kO0GO$^ZQCiw(hcI zazvdvVJbn+F+hFEhu>Sy(prZ?Pc7v`I=spU1dO+`S(3b}NC&55380-T>88CbbjP!%;17I`@ z#?FF?L9lZOOy^Tpn3t;aBFf$DHH*J=7PS&xRc24D2UR3NfC(@G zCcp%k025#WOn?b60Vco%DxUxi9U3|`bZF?%(BZw(*@SFDHX)mkO~@u>6S4`}gls}K zA)Am*SY=IE#L(fI{OPkMpN54Z_}ip&XXJ$-FN880Itu6Jlsq%2PHbE;qUT)wq<+$b zoyEdX!VH~jDmnE~X=*3qQIG$YlFomgr1OL{h72}~xQE7NP$&%l$?>Sff)J z*=rtok9@DS#%{7SFR<2IjY8OG%cpjmB`5tHTT|;6De1I-e7NkA&cK=CV{}?~tr~lb zPD{%uNvBpGdfOJ`mFA8`?u>U7-zjAld2CW<`3x)^1BX8c9cj>c5_I)~?mp0y0llX{ zUq9#{00UVtcoqx|g5e=Bk^|d+0;72_b{6IQ$li%2>qMQ~L=SM{5zx9~zx1TTc?b0L&b(pf0Q6HPQdrHO%6 z22FIYk?2@2A(z_AeTv+5v)nUB?kHJi(WdGDxljzH_`WHR|p9wpN{| zZxyv7lXU(qzuYLt#?;8INpAlxw+5SXNzN&^q~nmKVQU$Lr1Mh8H&Z6*#Pf&7mULRz zjgoXMHmln5b#Wi3sJSC$7I|z^W(k0<9iTf1dUk@|5a`LX59gK8< z?NKoL4j7Api8$Dq0FxhpU7cX+AlQ=x(;tJqU0|jg>`#H&qu^i7FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x zDoB8m4kaB*I+S!M>F}=UY(h36n~+V&CS((`3E6~fLN+0rkWI)YtgcC8(n#^t`yh(xSlV)k?}ipbZV@M zG;Pf4$(g!>V^%G?f@4-S{e>DeV$@xnS``Cxrq0pvNIl2a`p#DA=$zmFTiG3*_%#OU z3|-i)TK}O>^ikzRpSy0cn*X_M*}v?{LrWiBy6o|#YJAfuN5^WLr4GGfDyriY-6>rb zd2G^U83KJd(Ek${$b-T2V5k6wFMyF@u)S%kWYhu1R)Gm8*x3vwU0~N5Fy#h&TEMgi z>|GCLykP%EFzW*cw}3f6m~RCO0dRN+=y(-$z6QEp2i?0t&mPdb7xe7|{cnPS{b2B| ztNfC(@GCcp%k025#WOn?bgJ^?y9T!q9{NL+M{s-un~+V&CS((`3E6~f!iqOx5l5#+5nR;IMg6t3 zTX-SJIUKZGrp<*=Mn|WCvvW$F8#G@kOe=*^6UNK!oU5NNDYvh)s3bw>x+R;xKB&Od znN|8@A>K?!$5Lynt~ok;#&dMw?ig(4o}&|xdXBBl<(H1m7i-u3Z`NCNZ{{X=Gc&A) z2keGLYRiM8_TFzdEXLmX`g`Z%y?0n=-ZSsX<@4@ud}MiJ<6o*x$JrX|9Q|!|@@@86 zmI1{kr`xDatB>6=F-J#8M<@R2RdaN#YFpP;J1uIw!zf)Ad2G^UiGiUw7*2qZ55V?L zFnSP-CBekUU}qPY>;}71VCpE?(*vfDgS}}ma}w0t!Uo!j*7xB}~zofLv3GhO-y~ju_*y1h1Ca&{>TC@GoiT>gP-FYMH-u-2YIH?vYYU ztxc$kzBQRY3Fo4&0sL#J&#LuZBjq04XRM5LT!iyw!f6a6hzc0*?f`EnwOY_O^nV0NB3+%m%^1onS5m=68XGFgV->I`)CiH$m5a(ES$Zc^mY; z3;GU#{`bJZ`(W@xF!T`^J_JS%gY8Ga=qF(8Q!w!v*m(?sPV#dKItB_lOn?b60Vco% zm;e)C0!)AjFaajO1egF5U;<2_@(EDT;SwY+LE;jmDqDiYdsXvZ)fK;2HBUmGgggm3 zI)bAk*o16CHX)mkO~@u>6IQ$liwHVqMNF$@>Ckj^C?^_T_-_!7)>F3<4dps6(w!WPqw~AC`3~JV(KQXFp3*;+C zX^xoW)+Dz^{RPvXFPJX9U{Y)J7tHF(nffQpD%>HH&8%wr3pMJl@%)_0nV^%9YL2Zt zxJ~*wn@=ao?&mb0EAHoXpE!H_#QYqi%&PVOnTVf*Lz5m$8cd!9yL!P?AJ~%t)2G1R zelRlt_GiKDS#WR=%ngD099Z}X9L|G|UqR<@pzC+g{RilIdYh#88PNAE=zk6jJP!s} zgP|9|@LDkPBG|qTjBWs9FM)|oVCQBqxfSf%2BuyHd$w`O-Vl#KA^Q zv^}PYjVm9Pwt*BYSHl!>Wh3OcR3 z%vZ`dW(Yc43|HHQ067nSENyxbooEyO=WD~Lp*@SFDHX)m^ z;!Rk@(5dCf8AZ@CFjS{yaDB5w=#22U$yzI;p`&nePRUb)>ZHMRxuG-adHtjbJBx*> zgef}GmzmN>0}UPZHaWhg)@Hi2HLDneQ){bHZ`ZfA>O6g`TpFD-H6wH?Wawn1oMRhG zv`Rzg%UG`LhR(p5;v;lgcdZ(Ggw9alDnqAMZ8}s(7z50Yyg8VfuT)ccrzH;3bt6S4`}gls}KVU;yu5krS^3gr}9S~aw^6u}E&T3rZbG;|hnZcfQFgBFURyH^U+ z#fPkwQHIW_e=aGjup_^&bV)}#Y%4|W)I1jAi>y=p_z#_=Gh6p{rr6W@xBOD0tyZm( zTa(=WU2Y9FMSViwTGSQ#c80pbZnOyfuUWosw1YJ|m65&Xk@v{=T5IekOY;J2z11j$ zeYSjRw^?#h(XlnX__FkLVj7z@RJ-$@EmZuZX9~#@!>E1HR)2X#t)S;cmiA>VL zu}Poh1F+Bu4j%*^he78N(De!E{uK0l26~TyzRyAb7hvE782l0peFcWU1|#2q?cajY z(_rj7FmVR#JO?Jf2fKa%Q$K<|KZEICz}{cM%x_@-?_l;1aPaAuC3DY!`DekxbKvmv zpkoc_biXV;9aqcC^mHn}2BXO!D*J~&025#WOn?b60Vco%m;e)C0!)AjFaajO1eid@ z6QHNVl}KEP#Fa=@wi1aqtmX}?D}KXjo`gIJc@lDd1m{Pv3E6~fLN+0rkWI)YtauX^ z@pOzFR-y=M8iJYzF9co)(`g1%89kjk>7Hof_`RBl-=~S44``z0K~0>0NE3YzYvN#| zCfXj;=<-~2c@)vT#xP>)T>eDPx*4VEjQZydT08Z^P=cmYM9oF8A5 zES-o^cW-KS7pIt`bJ5Kyx1-~bdXBB-D>ypayn(ViI`M1F(iyt2S+)K{cYIVhKDSuS z|6I20Uw_FxEiI!Q9gEGXcAuM&ql06UF3UR5wE=X$1bQ}s-p!zIE9l<_23`h(+riK) zVE9!q@*3FwIvCvz#`b`TyF5~f=r92$zyz286JP>NfC(@GCcp%k z025#WOn?b6fyyU9M~99M9UVG4baZ&%bT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6INLh z7IAc_r%+F!r)8#|GM)4k{x;=rvy6^T1GE!ObgY+X-6AodXrlXOxo3{tF;^4K3p8px zy23PHDoiVdjHtM#GotJ)DYvh)s3bvWd=Y1Uxs{r_DHn9462v(=HC9E=xiPCJXG#^v zD4$s5)+D!@{z8qqXFNw|a^~oGq@H7I-5rpQPLm^Ac1Nejn}uHw{TZqF>sfMS!C zr8ccTcE`jV9jk4Y+7>ZQWR4DwO}Z>!gT8M-|F>Y^G#LC244nbP=fKGKVEYeX^hYrE zGnn`V?EDo>{swma4yOJ9d!7zRrk??Op9M3|f&I^e+123S3t(<7n12y0tOJKPfR2rz z(+9e?fNnqNX$8Fj(6NfC(@GCcp%k028Qq0(5k^3W=+bxC*JtRw40r)x2GG#cx;5laMDNPeM+Q;PePK zA)Am*$R=bHvI*IQ6>q{Kj*dYQR5Yk)n5by*LQn)Rgz0l3l+n?t=j@yb%?*;0&ZtM- zlsG#jOwpOxK${itYi8ZZL&L`>_N6!eei}OC8ai#`89I|QMJFKT99!FOFm%2?_i5P; zouO=TLnnS@-<&ZGo%p6vQ*?whbmGpdrs&L4n_e5G?O1FUHQr&ACW|~aX|n7ALwmvS zJ}~ko*uEc(z6HkK1{3dsod>|=dtlf5VCqA#=OZwE2<$x!W{!aUpMcpF&=*ocZQ=q3G^bUZ&Ea*QA1_r_45E#mV;h(@r9&A4kLnm6Gp~ERU zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_3KF29Lqms#4hr&7&=^&KYiBZ)38tkf17mfYIz~Z3!#jL4#(z{HZrJQ z^gTRE%^CI2C6i<7)Cpq=cFv8fHe#iiz+(m8d_#7QHgQ8}1(cxihXs}QJq-dgYHDpl zRrIYvy;I*B)!X!~>C)D$F3|s7qu#D>Yt?!BR#9i`Tg#>G47tS+9D|xQ=ueDl8$M~Y z8C8?qn&j4~zhL_F1=FP$OzOCH&aTSXIT0!5*y4w_OFJj=#dl@5bDGZ;w{yBroV|Tq zJIAgM-kF8LJBmffbenF!a(vD#Y3%eJy_TGVLz5EA?_l%~F!uCz$;2~Y=d)n)Ik4+_ zFtr-&c>zqX1$$ovGwZqaBvft+YIKnf`x71@XMfM2j~ofuAQJe1bTLX-Z1EE z1N{*&@CF!c2SXiTI0{DI0o!9>G!DiRVB!O?vva$YbCL(Qb9@fR=P&^#zyz286JP>N zfC(@GCcp%k025#WOn?b6fyyU9Ifu)SxcrFAkE(3>5${sXyHr>FF4a5UZ9DgyESqAUQNXB)BX#_ z@HEI{Unz_uk41PsdYaDVd&=eOXe9|c6L4$B6>;hy-~4r?*Lpr5@3|o{=sG^$@So}E z+&YDh&bSkFo{&F`K@NwxI7w%Qy25U>2>q{F?l;=O8lB3>Uh~L%)(io`|D^ zLz6DcVK8|F?D_;ueG2w`2BwdJy`O`bFTnm2VD?LJ@GCI)HJJYfEPM+Np9URS(0LYg z4TA0=(31nbKY_kH=synz3SjU87#aq{O*>vIBOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-6SfQ}9w z9XdL6bm-{tX6bA~HX)mkO~@u>6S4`}gls}KA)Am*$R@0^CM@FUa8>^FS(Q)6LJ|CJ z(z-M9LXa0i866$vzu38%sBcs!9j0s9Dsq4Qq$xX#g(zW&&iG2r7(e_NFD=1wnXi5< z#!d=4a=6a;Lv*T2(77n)lv~hANjJyV_bmjSReLv;UC#kK}57B8lV7^ku zG23d@c))Pw&atQH;MkG1932z07D;u;Z87e5NuC^(T~Ac7ntY zX)t{f?Ck|JePDkEhEDbr4IKjw9VWm8m;e)C0!)AjFaajO1egF5U;<2l2`~XBQ27LC z=x_}Z*C25XQkAVi;?1ggv+9c9tePhwPePuAoE^d05o|&>A)Am*$R=bHvI#5RghdP; zlOnjJpG*49bX#~KD1sNl^tlkqXy`2DNfC(@GCcp%kK*bZFr$bMNo(?@7dOEyoI-8J9$R=bH zvI*IQY(h36n~+V&CS((`39GCLi+DO*nLmA2=F_uK1b>?}?+m;UV>HU zP3Mw2Gvd%p_LBxWI_hn@dsAJGPQ<9YIJLTqQ_RtsoLM?Kspr_596{;mbT%I|b`}p{0*5UH14=btrI^qhnFq-ZV~RmJW_h zx-4&kh5g|0TcG0|&=~_=anPLrJs*JHPSAG{^e4f<$6&Av40VIy6c{-Qw)cS1<6tZe zCQgE#yvLovtZ^d*gptnhrq!cnEMIL=fT2xaJT?Eeg~a@z|nC% z9i*c}M~4Y80Vco%m;e)C0!)AjFaajO1egF5U;<2l2~<7-Iyzj1#8pUKg;Zs$ka){# z-m<#lx2)z#$diyKAty+1f&`n8O~@u>6S4`}glxi!H(?P+r$!N6)XzozwX|D!At-_u z!t}Wi%IN4cKs(Vy$9jp@EfNEYCc1Byd*;X;b2ZVtK%>T^D@^mH!n9JDG-0~h(Ybuj zl5+bxi%Ju8G#E3MDSNEIcg>pOM+3fb>W=Z^cQpu$F5(DP(YFTmPJL^vNN-Vj(wJ8K(WcbQkzyEyJKR4j>TqGTM82rbZ~4^ zWLX8eoS?fI^teFp8qntk{Via?0|wWFAukx-2u6Hh`xY?j2V<>ZA^>*o0FyzmYbTfr zfjzsxbQtVy12Yk@{|zwP4i0vJxhR-_2Q0+E;W+4cA9Q{Qx;_Hkhd|F^(0e2(1s&ff z6m$%gUwl#VJGAl#U;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qq0u*#8=upt1phH22 zH%w;}vI*IQY(h36n~+V&CS((`3E6~fLN;NQHDM7!hfDIO&ysu!7K-3+lggdog&;44 zG738N9Go+up+VBn8TF`}5@Dx=5jvy2l{%O#)=7QL$Nx`B$5Lynt|>Zg<4HPC=u>tq z>cwd~Gt?D!qebX{%@%{rXa{R_DkFQ%Bkz&#wbs~8(v-2*>(b0@NHaEzNBTLoR{yI~ z(rNBHUv^1nC|g|8i67ZFXG}?_rDfC<9jk4YIutWaWQq=sP0B1O(03H{_ke-pU@#4a zPJ-cHFwzIMXTazwFxC$y2EfiNm^=%14T7m5uqOwme*$~+VCFp7UjVZgz`p-fRYZEA#oWJmmyWzG9=!tnm4Pi_|2+$67nSE zNyym|oE^a?WD~Lp*@SFDHX)m^;!RjY(lIN7tNOXB-$=cM7lI;qAxxhOp^TCa$L5qa zGN@kkJv_?J8THR4lOyZY2}22j&R8zal1VUW7EigFBV{0t(5bO1(w;G^Cud5@#wed) zypKt4HT{Jeb=P=y&P6My+;&btiaEBnv#@i%JKXYrSZ~$Uotvb(bJq;3;Q_m0k=pX$ zsJ-{w4U4gNzW(01c<&w7nfJ_la{0Xb8y{KT*l1TCTKeeHWsfgao6i-ubGlEQy?tVK zj*xav{B6UPc23cq7`D@P+9)j+d2G^RX$3<8FuVhd1i|*5U^E2Ac7cg7*x3dqBVgAX zV5%ML=>XGFu=gD>69fC>U^W2`egNh=!Tdq6kOYT61|6S(&QC$tXQ2BS==mJNfC(@G zCcp%kKoulFJBM}-?Ht-Uv~zg7bT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6INLh7O``< z9)J3*$ERJP2>v$d+nIPF$P1y2cFufhooM1DVMOL zm1O8lV3f^5$Ti}|AM$FP&kpko8*dNfC(@GCcp%kK*bZFr^A&KxTck7-2Gx%x@{q-i^gg|UQ5Iyd6o7*~o3KN|3n8~M`f`D_Eeajcm^ zlksZ94eL3#>Kczly+z;7P-p8?ZwzYIpg%FHZ42ZpMk$V%@d)4FTb*pqbPn?~t5m_F3{KTFqha9~nn zc?V3!z^*u$N`O5dfay-K_aK-_g8d(Z*)DLf8_cD^{86yb0}dYt9VbBNm!Rt_(ET;& z`3CfU3;Irj{_nuR88CPb41EuVe*hytg6%(p(OGVMQsa5|jw+5S{uF$uZOWPUhin0vQNl7`!*4G0=XU*E< zQ)cMI^M}SZbee+ZD<|WaWfV<&45ceNI5MfSxWM!pu-6S{TEKn}m|YJJdcoXAFz*8k zTfkvI=y(})ZUA)Am*$R=bHvI#5Rghdn`qawJbpKJOBZi zPZK*I&_v6FnmGTECi))M#KA^Qv^}O##_46APcIt&F z!O$s5t0_gjx%@r#&~R$6^{B^7lyoe$Hj{dr^j>P^coq!9skPOpx9i(lb)LRe}Tfu8R{?+>8wN6`N>82AMY{tAYE1H-?Akw3upr$ds_XTaFA zVB$Hj^La428ti%jOsxfbUIf$Yz}^jD<|VLy6PVo$4sHc=+ra$GAt~t;wud-JhmsBx zU;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qu0+e(p=}^+4q(e!E_e^IKvI*IQY(h36 zn~+V&CS((`3E6~fLN;NQHDM7+hs*M(&$4_<7K-3+liHn`7lOPH$|&hHz&p`I$9jp@ zEfNEYCc1Byd*;X;b2ZVtK%>W_J52MX!z6_i4Nl5f18bI)Ti97tlA&`=kLFr_%|!P< z3LnQZ@>qoD)N`s%&)GAco-;W!baK+nu{He&J*TVxx&PC8tFFx41Z8H1)$oAbut;rr zaMa%W?S{qJJ70hAT)g)V>&$!RJ-K||{f&<-Z*2U_vgtTmW1XYFtxmqpKFczo*koU+ zO{`@wPH;E`I(CE3J)mnZ=-vl<-UPk-LEl@T|7|ev zE*LxjhTa3i?}L#K!S;{9=pit67)%@iJ3j%FpMqVVfvIC)&*xzJ3$XVDnE4Xy{|d~0 z4Gw+-=Dr2el6u}E&`dkQQ)N|@NH)ldKgRVAG>ZU~3DPe-nU*^n+ zLNlpP8tCU-8k`84ayPq?ZE$-*UkL;T>rk^tu7&Sh} zYO|`{PE+ZA4vtKEEI)&;UqJV-pyxNx`#b3S1N1-rx@6!PF!(GOdJYUf4@Opl?Jt1Q zwP5T;FtHBo+yEwD0=qVWsm)-|RxrH{?0p%`YzO;a0kf}ygRg?b zehydUPoEX}^eYs>-zJT_T3!h9LMWr3!+ANS%?hd)eGgyl=3KsK$>exCb;4AFn`2!m zru5N(ujA~Ts+*m&t1@nmM`}5?*0sB&o6{28Uv@XA`CM@~r~Aa&+b8Pg%u?G9Pr%K= zkx7T;ebDzI=>G@|90G%f!O#&f{0SKO6m0(tj2;7HpM!}nz|Iq3@=LJmD=_sn*z*mT z{ub;#4Q9Rr`_F*cbKv0jVD1Mn|07uV865rvbQD161<*ANx|?=MdK{p473gz<{$?=X z0)uOINjE3t-bFWuZVnS*0!)AjFaajO1egF5U;<2l2`~XBzyz286R3OwbaS}+h^vpd z`l!lQAMqB|yhU}zZ&A&YkS8HeLQag}#0WMan~+V&CS((`3E6}dZ^9yOj!*;_^K&tO z4XqYl2#Vl^FnunBGP*hQrE{W*gN>SKdrT7>S7@T?De3F1GH9ZEjYP+KiPkL=1B%>r zv)nUB?wG5I<^`GqoDV+ktd(si^6`Z`S|2vyOy2K7#TYvk-4A?&j)%?s={OF*hQwzeeA{bf+hBttbm%#Q-V01GW+X^POft@dd$?ag*D`4tXu;(=}{W{pY z8_eth`}cy`ec<4mU~WH{e+w+U4GzBxI^v)+0lGc_-JPK4Am~kkzK=nF7Z~UUgDEg{ z6b$!(k>g-{8h%dnB>fyi<=0+R{Kl;O0hj<2U;<2l2`~XBzyz286JP>NfC(@GCcp$L zo&fzE`Z@G-=;zSS;eFECgls}KA)Am*$R=bHvI*IQY(h36n~+UdWldPb&*6&v>9ZoA zeuX0V+oW-4L*RtSu6}C%+I-|kW)WtB2E36 zevbaCaiu8#qaOd~f1agtt8VpJ)Qj60>TIiGG1v@h)}TL;ipT=_4x`Pen&j3bw?_R1 z)1NPxF1=tdBew(+LuJRv_|uvyfL+ZpN#yU`-_zh=4L zXa{R_DkFQ%Bkz&#wbs~8(qysL>r%@lOD7@)9b5eM-O|!YZ8%+aOK0Fr@hLj3yH<@o zMW?0ZDoe+xHr;cjrGw;K9=wL7gCmnR%U59ZYcTc=nD`d#JPjtl1G~C9Q+N;{SM~;01Hp=mK=TtbT~n0Gw5=G?lqvt4SHKZp9l1>2LoO( zxDgEb!0;9@;s@JX!Ds-C?En)&uyf~bY3U?GyE#RNmJSnO0!)AjFaajO1egF5U;<2l z2`~XBzyz286R3Owv~;)*iR+NK4ynr4A@OF_yjgX{Z&uBdkS8HeLe7rh>5EQ`+VftJMWwdk_a&k_|Q-c=DsX;4+;o?J1 z=DJ$cxqQ!(vPwJh>q-!HN}6f&DAbzrm@|vY^#4rg=V_p#Q?jD-Z{6Xkl~NC^pIY_r zx|*Y?D=sNI7q{h7bQ03ev336nMQ7jBe@vO86VD$SThVDRCK85P|=~HLq&%-O=lCb z3E6~fLN+0rkWI)YWD~Lp*@SFDHer=DVG%`#i}I(>qI@bAir{aP(w%`9g1ivQsOZ#5 z?L-sD@6|;7K27X=KoczwYU2Dun&^9269*eL(e{`|jpw4qqloS`mkgZ|Rp;UpIqK$e zJ?HA@Hz@4L$uT7uI@jaf=)#M6rMT|L^?d1sl$=p-rlB+L5S_?)Lv$v`&`C)-$JX~z zSQ$##Z0MW^z2NNqBR%KqUGzyz286JP>NfC(@GCcp%k025#WOn?b60VYuK z1gPk65fT?6aS>9LEkfeWs(G{Oir=i7Cm~Nlo`jqo!PyaPLN+0rkWI)YWD~LpE8c`f z6rCDHa7{ng^w&~u;e{ZFa8PcUHWxw}6`clHCz|M3FVVV1VnESE_sw$89JynzCYl#$ zw0LxbX}&a=q%NXih)Z!%=i>e)<<@oNteO)1oa=UM{`#Q2Kvs&{(jN=)W-2EJ&v=T?6Y>KFo8{6lopKG)$w)iL zHnd@nRCInm9xl70({Z-OI!8Bg>ZFNdpJf?PY_hV{rq##pn3$qdD-XT3$v6Q;2ZttA zmP26nFgSPw%zXmpKLrb)fy2i@$8pe^23;pXcQ5Ga1HBp0cMA0PgMk4sm<2;;!SEm$ z83NmLVDu+2mIo8(!OjAhya09$gQ=!Hl06PEy$bAgf|+Kp-vwsZfP-!@*8=7}d!(XM zSigsg4iz0Hzyz286JP>NfC(@GCcp%k025#WOn?b60VYuS1gPjx(V?P4MTd$G@0iXe zWD~Lp*@SFDHX)mkO~@u>6S4`}glxhpYr-Om4j1LqPNAkHXs0NG7s9l<5Xz|N)N^vq zgr){vt)$dViLz6IqcZ`wrW6ZjG5j0L7>^}*m4Z&y2|8`#2|ANALnkNQ99z>V2s%9< zbe3Jv8OjzHbmB+$%^6eBX*w`!hK|K%RhvGXh@gW*lOoGYU||zDycu-(L1!!I3V`k% zpeG1=cY?kU=-&kf!eFos3`M~38(^dzZ0`W0Q84xnn23R$aWI(xyFLI@onX&FFr5T@ zKL#^hV1GB5O@V_)!CViRKMoes;P6S%@fGO&8iJ1N8wxsx$}hgC_zhb5126$5zyz28 z6JP>NfC(@GCcp%k025#WOn?bgJOK(iT!O?UNL+$cWlNBFt7_h=y5hI0=1IttkS8H0 zM{sfkn~+V&CS((`3E6~f!iqOx5kbeSh-tMfecFt`pfW;bgx97b=uR20??Nb}pu?d# zr406=>(I_V3!L_tpR)7V7dkD^?;f6 zV80j4ZUhH?U~UVT_k)F2a5w-uUICr2g09y<_v@f%H|X8dCjA`W-Zl=egTKoN|NdYg$CLUmpUW%P6AOYcMz2OBlf_LwF% zuFyo&Q_|U4WzaZrEecFG)Af5(! z>??&Bc`U;7u?&@RDLYz8j?QR-W|U7;#(%DWnlnoiv&KIB$#}H^flu9$Ui_{GVYSvu z5l5(szBQRY3Fo4z$&+M3k``oC+`+x2a&I#1s!>TG>$QE%0^GvvP&i@|14vj+W% zQEkH~jW(ldl3SD98ub@Uf4*S4^nyvP(O)pDCui!PFsl~*6J}M@U#L+lG(g8A-5gtM z(_Sg)Y}}GByP$Ke0Xj{Ym^!AQ)4FTa0G(M7bdF9$(7~Zek>w50*ADtSz(5oXz5|A0 zU^os&5@7oWV6+pA9Rw3eu=8Uu*#&lWgQ*nQa}-SXfW61TOd9Mz31)l2!9FmT0rRK8 zLO(b>06NZq&U2vad(iy@==l-!{tWtl0sX&%f#1O3?+|oCe^Aij039a41egF5U;<2l z2`~XBzyz286JP>NfC(@GCQt*M;bx+; zQJoZ+M(mucpVUv9ud`T~N*JOeb(txBG*Hl?pd*BRwxxN2-DU|$H^R_ueqh zFx<(^zVQ5de&=}3ndP`I&fv_sV3;{GUNEyeUO2*xXBc(%?4d@C5TKNR5dzj4r9#9~ z1B4nNlmGz&gc6`&fKsE@8laY-5hB#{J_~j?l+dP^0ftS_68 zO~@u>6S4`}gk!7;^9VY0Q|P8p(ki2*r3hXKdb#8{fy}InRgwglm{0I^XNQg;)DKpqjYYM(6QVZsWVEW zbpEN&+9{XAcQBo&T>Z1$8f=RChQ77j*-lUw*o_vU|2NCmjdrk1r_#UI-2Wc=UTc}% zBn=m9r7pYNF?9k`(y_I!4@grdd2Pd}nL4f6!-JbTZR_tdb&P83bN5O)W@+lkfnoRV z>^F6AXwqnD149WgdaUhR2SIQ4W`pz&t)*v17>@{ zTn6m>33U7dI&XlkU%}d2p!+w_^HxC8`!-nr4(NLqY+M5Rmx4{p!N3YIxDpJl0>kft zk@vyq+JH25V(S7lb!h4^0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqPq&YmvAX ziEEL@*jgmsv6^?RF8Lj+c@pv@M z=21l3GDE+sbN3TD>}GHu<{`bEN?|BK)X5{~%(+XXSqPD)n2))mXy)|$KUGw8tcp}< z%ExLhYRyF;FGBse-6`XQi!7)lTXP~09;Etkmds{3;CnD_}TU#26&d!Uw zi>~OjK46$m@6C0p^{=`sq+%G)V^;Iu=gu&W77H-kL^FtZKJ2Ekkl*cSpFyFuq3(6tw=-3Pi4 zfS!Y(_b^z01oRyP8;^tj6JXOxFmM_So&iH&f#GvtQ|r025#WOn?b60Vco%m;e)C0!)AjFaajO1eid{6QH9*M~99M9UVG4 zymLC6kWI)YWD~Lp*@SFDHX)mkO~@u>6S4`%SQF-Pbf~9LPobw}rk*mM^c4OzW6PH9DtX(2-C6ppMRDsl}ArMrVx9mZ2P-e~-~AuA`HXdXBBLWvg^_ z)*m@tbVsM3AD-z5}}61#6dp?xmn-Ip|#h)~^J8tH8$h zK>zz-(^@dF4h(((hCT$tAAymN!RQ7s_9+Z-Yx`E|=%jaS zNp!5mQ(cvm2u0rA}q%pP%iMOlf z?W#+DyK0_#=+Gcr?jQc}*b^*XmnL+4iTO3@9S-gJILr}fy@DT5k1t!oEN(Sa$|xy*R)Xq~~Q z=-}9-$r1w7yTG0>m}v#G5iqwO?2Ceq!=UpB=sE`09tYhgK+j3gdm5}i1Ny!K8_$9M z3t-blFz_`P{00nt2Zp}~BR_!AOJM8@82=GWTm{>&fytl2j$gpk4Y2E1FntT``3=mx zwM{bn_BLtg1;waA)Am*$R=bHvI*IQY(h36n~+V&CLCi;n8(oJn*8yzCZC3d zBKX&&b63s_L0$+&G;}yNr?8Phm7?>d0cy^GFXxVosZ$}01=u+cui97^ieWq!;LWsi zH0S4RE{&a&kz$Um<}~b_4&OWfi}i7R*v%ty z*v-=utcDlthFNOeiv#vPZ#T@r-s$>#r{cX&SttH&+MBOUd%pVR*Q%@S%ItZs%$xuE zJazGv{B};;xyw%u&(5*ftZM6tVc9u2Hfgae26Ha3ZyD%V0XkQLu2o>|d!YM$(6biw zt^?~o0DT{VjUR#jkHMx5VBk|QxCsn>28KTeBU`}e7hr5F7~c*ic7W|)g2|m=$8IpS z2khDlruTt82f)lhFnbux9Rd4}fsQuNnSh<+Iz>CjKs$#CFaajO1egF5U;<2l2`~XB zzyz286JP>NfC-d70oplSf5i1iTz@pi)*taU)x1r0$!}B5laMDNPeM+O;M52cdg)n|Dgd*BG)1`Hyi9^+zXns`_YZhpt z=3mm-S!B>e+cJrk)e;TsCAt;4>ruI9irg_(6N_hPqW2k1oPAakt6|irtLq4YCJ(P6DoTM}42%R^iIb@J?WNwer znV>GP8!bZrZ@+ny;f`3hFcSc?a@_9JtfwG)a?19-A~- zHh|tw!TL?0?=!IRbI`v9Z2AHWYz2ec!O#vc{3RIK2}XB=u{~gXFPPW|wjThK2f>cR zVCo3ibqq`&2YXI{nUi4lG?+UB_I(99&V$Yl(A5dnra!`F*;0u2`~XBzyz286JP>NfC(@GCcp%k025#WOkfNoKtqRXkhlhkYmmm+8YJGW zns=)%`Q55{67nSENyy<593H_YWD~Lp*@SFDHX)m^{ioP|df6=!_^$C4zy0bN_GxV>^)F<_A zxjIeXD(Yl?Yq_(XAh(K~SYuGr2K|XqZN_&RZAR53wMxl7{etPv3nsNpf5EJt zpQwL_S+(fjVOBN$g)+5&n9e`-Hh3f|17r#o1*@En9l8Ov4-jRq@`nP-V>Cn z&RbnaM@`jf%^n_H)oI&qwpQpyER(=TA)1{nMm4BZ05zk!jrf|Aj9|W6Pz(5EL z?g~m#Cln5HlnzJfFaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x(kDPshoTNe9f~>> zb$Hu!HX)mkO~@u>6S4`}gls}KA)Am*$R=bHjO11hs6DCrV;VY-Njaw6Hadn*z^I!z z<+_Pul(*lV{2W$LmE1%6Ws0Gsb+bm5X3f)&S$9NmI@p*4{Wrm;J}^+TLo(<9LyN$$6O1edqplrN(TOeFK}DzZ%P(pS zqO||`3orpDzyz286JP>NfC(@GCcp%k025#WOn?cLJOL^?T!h3$NL++8#ug#*messv zb;)m8&6AKPAx}b1kl+LfHX)mkO~@u>6S4`}ge7mnJc>@4BDkiXYx>J6x9~zx1TTc~ zb0HK_(W!!UqKTH(5)JDmx)n{dJu3H1kvpbpV(|=(7LRT)O_v7KLSfQ`X>jjlZn1Tp zS%vvI+EAZ+zc}6h`Z@PZ#xdH=YUhdp<8lTUbZ~4^WLW{mR)XJ_OSrfju9CnGIm}Q!uv)?E4IKG=a`$&=mk{w}I{;=xG7HA+UZI z=nI35t)M>wHth!kQ80K248_3kQ7{q*qitX;0me^(iFUC4ECiiol7fzbf({d40!)Aj zFaajO1egF5U;<2l2`~XBzyz286DWNG6m%%)P|%^ELqUi4OJ@_Z3E6~fLN+0rkWI)Y zWD~Lp*@SFDHsKg+!aRZwm*kJ1CHWLA6v4kHl{>);L0$+&6m%*%H)mKggQTG|;88Iu z!cGAbbfhnH&!Ylgnn(089Y3FvPC=5+_&^w|C=oa zo6!!I=~Vjnn)}})-)k+io1`gYt< zpyMadnFU?f!P*??z6pBzKyOWpWW59QEdm>zpnoyg%sPVFxdch_`uXBU{_;{lyuS?TR26h^b0Ubev4NA0!)AjFaajO1egF5U;<2l z2`~XBzyz286JP=*Pk@pRmmzT(5|<&3v1LfSSv7A~UGke%^CaX+$diz>BRD&PO~@u> z6S4`}gls}KVab~?kECN(1XuNQRlkvX3oisk@In|r7eWyw9gfW@Y-CWS=zM8_oipIe zxg#U%R0u->g3e$r&fF0&Y37W&nj>YPDtxGQlpj^Xs@pX}RrIYv{foXes!!y8`c`{Rdxm+ zp@Un3z+!=%x(p9+rhpapd$o2cY&@jSlbG^BcNwL=#7H)hd^HpY&;73 z<6u)87)XG@Q(&ka44(xfNicdIjCFwVPB4)I+rI^qU0_Eym`a0Pm%(%o*wYJUGGO*6 z*g3f@?HrEKVFFBm2`~XBzyz286JP>NfC(@GCcp%k025#WV;}+AIkaO|(9z{TGblsglROP#F3jv+#W2M4h|$6ie9A3NmztG0J8pJQvHuKNS3sVSH{; z-b_iS%&N!{H)i$xL@5Xv6!;*AxY*J>q0p@-M`)+}bnlB}t4$!p-taXC!#h}LpdY6Iq zwVy zP3OTt2N>)GLn$!)Eg0znqupRE4aP5ni5{@M7ffcrj-S9(7VNqXrgLD=O)%33W@|!{ zIS1Ic2z0y)I+uX1rC{xH(7gimtPDv*$Ga*-L#OnsFKP^;wEy@EFaajO1egF5U;<2l z2`~XBzyz286JP>NfC-d50UA0qbZF?%(4nEjTc)!K*@SFDHX)mkO~@u>6S4`}gls}K zA)9cFHDMk@himf3&zgK17K-3slg?ckF9dlZ6w%O`$D^zeluL2+`)A$1Hp~sgTDjcpJy)SjufC^$9u3rd*D)nW1mX)F<_A zxjIeXD(Yl?Yq_(XAh(!>V^Gru{fSX+#&;TRM%5&@Cb>20FPQ%Qg6Yl+Cbe)$=b!qI zESKMbA6c&cS#AwBMI9MQ$1VLFTSE$x&c?IXMorRb%^n_H(y3`2An6Dx=`?(M-xwW> zT6fEEueoED19#e-CMmPXW0NwA2lTE1>%E|FJ=j@^Ce9j zs@6pFs~Sa~+aix5+LjsmHJ!Vk$U!%E3p)2de@JDgQiuW^or1)gLgbsf-%|+@M?B~u zf3XqLbSh#0Ja9BnC0@EmEgFR{<=`MGxs=<4s_0vT`WJm`ykqE$EmJ38)Qy~S^$oc- zN<(L$se}Cnn?--!V4I*Wup2Ex|8JK2jdrk1r_#UI-2Wc=UTc}%WLZ4JT4^;3VV^9Y z+HDq}lyq#(%{!&3vt<9$D(mC=i;u_`pPpbfykIxXQtMtEu=jbpVGj0A*WWu8?|sTT z@o&@Kd~MqE)i1wRU2Rup&wFLw{MYBHtq&Nd(|dEBYW=JJ{VKlyF{}CS^XLELKi^oh z$85FhuTR1+s#t{dpqeM#>*QEu9IY(_?45p52ZttwmTy6S7ueJd2GU^gG8pOs!@Xc6 z14e%WV_7hM9ZclF_M2d`5A3MfDVcJBU5mi96YNgP-q|qE2YbPEOM){UVH#->Q|r025#W zOn?b60Vco%m;e)C0!)AjFaajO1eid{6QHO=QHP=qMIDMdymLC6kWI)YWD~Lp*@SFD zHX)mkO~@u>6S4`%SQF+Eb+|Nt{4C9=xo3*pF;x?bXJ~YJbdPDe^q8ctqCri$Gg2q-lexu~c4if1={(S-d5~i>-2JcI zd*HKhm=6?bM(CxgKc=7axbE|e&Lo{JL-{!)Gf5{P)f`)EAN-v3mXC|>=d@fdvrf@9 zn+j+)lPukeP4<;qv-I>+!}D`6uShyUkLf?`PxxUL%iw+v4o!M20Wh=;38h? zd;@yF1M9yBeLsMWmq7m&u<1uIa1{(*14BQ9;a|YW4KVsE{G8Y=`Z*k=!vvTB6JP>N zfC(@GCcp%k025#WOn?b60Vco%#y|q}bGQPDE0DMXX^gEv;{B?5zv`0TubL+zPePuA z93R2)5o|&>A)Am*$R=bHvI$Gxgn9fNgCe+~p9}g;G+THfD1sNl__+{@=;u^&c+Ri} z2i-5VRNSkr+^_5u#n&ldjLu-r(EY#F!|$o0pi}SwozWC@noA?-M5LQzYdgA23OYyD z?JT;W)0@sO=(HZ&I%QBnr`Kn`SH>~h%&Js-?%g@Kp@UKCTMs%u1f3s&u8+am4WRo|(6b5jeg@Wm4*Ir$ zjbDKNtzgr3Ft7s*ehG$lg5lj@WDgkK3&!?=@dIGuAlQC*mo#*eM|N?74kzd^0Vco% zm;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oh7JuK8agy|c(-&mA)Am*$R=bHvI*IQ zY(h36n~+V&CS((iu_nx8=x|N`_*s)r!$J}KYtp$Z=Y=3Igd!R`9GO$txS&eW`O^J% z&fRIp&38R3BCNIF8S;l-o>qEFH5tLm!hG{L!?JU5Y|>(hgUL3qBLSvP zfnDuj`YhO!1T*KsYzLU@1p88;<2%s#J?Q!Yti1%fuYjH(LGM+t{u=1}8EpIo^xpuR zegy-!z~FCS=&jw7;kUuaJ7Dx(Ft!AYF9j3J!S)qkawXWY3QWBRcD=t_+Bxa9yJ_c? ze&t1tL6r6%e*q@I1egF5U;<2l2`~XBzyz286JP>NfC(^xk|#hrhwG2H{)p?3#@PBJ z-lm$jsV@0#s(BLfB;-lRsS%tS!6sxAvI*IQY(h36o3P|fn8(f$ir{j7F6S?!*TM@y z5xfw_&xKG#J7>DIPBd|-S`*E$YGTa-P1O8L8as;&nrK@l(Xv`XPMe^`)AneUXqmE5 znED^H@LUY?D3-9J4b#w>1$U-=q#v)~BWUQ1IYXyJ^K(*C&aw4&!_ZlNEnRd&=Rxyx zYEof!P(!ELGhlvBxy_<>ZZHmKeh!XJnk?(VbUoP90A_q(_7gDI2=;9R9iM^D&q3D~ zu=Weky%qFq2faJM`Y%D>POxz|=-&f2?F9q-z~BKebPx<51|vtn=rJ&M9E_g;6DPs; z(_r!p*zpyZItO-L0Mi%2p0B~oH(>TV7&^J{Y3LYe=r92$zyz286JP>NfC(@GCcp%k z025#WOn?b6fzl^HLx+YA4ILUfG<0~cbT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6OOSa z%wyVCsM(gs)@xjG|~HvCeA*qiPq;dvGD~> z)V-*Q>$5dlJi5VDAq}Q}J?H)>l_MtX%n^nHX6QUn$*CMMm8Npk4ISwORpCReqx`56 zR(%YPP!)Y^Q2(NDjd#Z9jIA*`LrOYt$Y0(dO_c@uc7nRVZnOyfzgg}#+QBlNO8;JS z|9j+ntz~wTW$_GarPU~eeX@LNw^=gM&#~3)2un$)r^E5TS|8V6d_=zZ^aQKn1-oIE zTKD3Bz0cbXbFg>1{@$s0?^D)^f1CE^Ytx>ue)+ZPYP&Lf-YfIwzdldxzLbBAPQ&I! zgOAavs~aHcSZr3cH9agz2gfF5mNb~V4EFVaj;o;a8tD2Nto;Ra-vB+og5FzT{coV} zt*~U{+o1m)u<2beumlV)1w+fh@Cq=p5{#|_WAB0S_rb(kuzekv`~d9u5KMgpc6|({ zH-J5#f|*TV_A@Z|IoP)abOb=>wy>0RT){BM=#+i|M#=Bd%3pv9FaajO1egF5U;<2l z2`~XBzyz286JP>NpyUZq(%~{BE<@rnq%pP(iFd2!-KtA|w`!h*JPCOca(D!XN3aRm zgls}KA)Am*$R;d#6Xub0Op4&Dey-{_Q*Ysgpa@5y5G~ed(YgWIy>^?3h;Ca8fx=O@g$d7kYx(`&?3H#l1`abkrItrJwH)8Hb(ix zBDW^F)$|w2)P70lpZXih)iV7JNfC(@GCcp%k025#W zOn?b60Vco%N}m8F9ZEWsbSUXi(&1gx*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL4YV zkEFw8`Qv94kbX??&To!*=4RO?^$K|Cr3@jPZV|9$@afBa_xPwUzNj*ft%v-ZA0I##vw%z&L% zIdZ4XX_79BJT~dFybF4lfb~m3-*T{V1?XQ1Hmw2!?}5Sh!O&VTybg?f07gFqV;_O> zkHN$Su>Dgoxe4s}3`~6vc5MOEUw}PZ!OV6ry93O93HI#-9bwSf3c4a-?S9Z51wDsA zZw#zI3i{$;V;dYDe}ayVq4eu7N`9AC{sK&Z2`~XBzyz286JP>NfC(@GCcp%k025#W zB~O5k4p$*@6%tn=jj>foyk9l%S6%Y^Rr4g|Nyw9s<0Cjef=$RKWD~Lp*@SFDHetz| zFpr~CrU)+T=c4{{+AX{g7&T!WAm|MEa&EDGommA5IuGO7-1%MwvW_VHF%xetf}^7hbF|M) zctxZAkTR1rUdnAkRrIYv{foXes!!iMIM^8Sx$rgGhowK zVBj1Wya0wSg5j^h$Twj0J23V=82Cn>QUDMfwY(h36 zn~+V&CS((`3E6~fLN+0rkWDzonlO*0!*%%-RA_1m3Mz`=g)puzgd$oxl~O&?MBR&; zxISAGoiAzPP_-tSU)98#1)8Y&m(+F^Nq2`T&mE0PYB~cR6{BM96fj3;7_rfbaQO2 zKSI#SEKe0((CJO*7j#;WZJjcxpi{HQY`sU&G1|;(+y9%_#mT!~k=_1x$Pawr>TK+rf?^lfLVxaRV z=!%22ZJ;{=dQO4fcCh{|=u3i)=Rto5*whIIQef~~Fw_NxyTM2rj9!MI6YHU%!!bHc zfC(@GCcp%k025#WOn?b60Vco%m;e)C0!&~GBtSujOOUt(iA#{i*b*e(t(teMF8ST6 zc@pv@LsA zcOeu}(Ba&i!e$0lN_pvSJLUf8y02mwFjD94FMr>7oeCiekaGqTY3~13hr4GwR2=hw zM+NrK&yj<5h900Zntsmaq5PbG56~&DpA(U4j;(F?Ug_tYym6!Ge$L`6`Td->bC;hS z($9HJAE5Im4A8My%(im1`9niq7bovdDYD38lOoGiFm?@${|qL60o!kY$zQ>aTVU!p zuo_D~^yI^(+m|F_=Ee9QSpwkVyR)e)3(7guqctP)au)ZGjHGqvi(Ekb8 z)CdMPf6JP>N zfC(@GCcp%k025#WOn?b60Vco%m_W%BprAuRhk_0T9SS6S4`}gls}K zA)Am*$R=bHvI)mn6Xp?gxc>cju79UGpa@1szs%_lJd-Jtv&sV?v zT6MKunLY27dGlYNr#@(cPE9JT4r=H$Y&PHP-IT-7k)w<5(RQphtJ>x?<=rVw7I|#a zWZ45I_ktb!z|;Y->mZmu4E7uWGsnQ}aWHoR>^liM+Ck@8(3J#h&x7s`(9;QeQ(*nK zpsx#T><0a5u<0@w=mCShU?>BIe*z;}FnS$~<-qt&FwqCL*F+?f4zObpm~w($izCv| zNxLF6bZF=>0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oYmm4GiEEI?*cv3> zshW4HF8Q6Rc@pv@L#cwRYT!p;0f_GuYS!`kTR~02tT?27_Rz1q_G4$SyD%24k&YJOU>6 zgY8i;c?j%?fvKZlR~$^YfjtQ@a|+D1!_di{rJ-Xe{pyR7-=LMh025#WOn?b60Vco% zm;e)C0!)AjFaajO1eid{6QH3(Lx+YA4ILUfyk9z-kWI)YWD~Lp*@SFDHX)mkO~@u> z6S4`%SQF+kbSS4#PNAh$MoUW(yb#9Kg-}F8XQs4HG*R;}O>{3ZXrgVIM9XT4hV>HN zirn?6+(U~;8cZ{#!L(4AZa-wL3@~&Cd^xwM!jAm7!X+JPKQ0uO(Rs|o2U$n?Q8ntH zC=XD6sP5wkRnfNw^)LF?IJP~Vf68CdD96T>$*oCl|17r#o1(s86ol5^+bN_qfd#z=5lazC;mHGgjJDyHPDmu2Bjr*jhlijs<)I6Qm?BT&Z zowoG@JROT1np5-3usj_coAg=Efw>D{-$l@o0-fK2t}d{)8+4~Z&t=ft1J?I~z6{v- z6X?%^P1nId4h-G|Lw#VlW}jrl0Y(>rF((*b3?^J)`!Xu$tk+W_FXgUMFd`N4j zQWy%*bn>VcQ+jzj7>wz@1FoiiK1 zD!QZ7`hZzFy*JmX*1zhqkSe{QrysML|2}{IKkUlvd9Tcy|N1<&u5N&%W3^3EJ9nD$ zj?KxtQ@Skj*rdzi2VI-N+9uH540-~fcN8z|ej$90enX zz-SDN9R=fYFwq9KC&1(>u%jJJodvs+VER1R(*b5W!E6f5eGB$=fsP+Q=Oxf}1+4uM zbYBHM*Wl=Qf2O0uSvpLB2`~XBzyz286JP>NfC(@GCcp%k025#WOkfNoKu3p;4jmmj zI&^e+-*h%1n~+V&CS((`3E6~fLN+0rkWI)YWD}0DCd}jLP*0(rLQl&~J!L%UDg0~7 zzh)5~ohpvcDRgwubg}V;g+fjk%CG74D?4+G?d!}cNYEL2p3QV9HIs_?sK8Gf`g4o> z9?sDzvntZSF{|e%N)^W_pIGG9B)6LWLYcZ{C`V^x=IFSko?~m+zh62!Evv5<-O*{e zTxOl3t2h->#j#JabSpMF@k^~)dits1IXXf*It_8dKpkh2+8oF)?nswK9-DMou7lnj zSbr1r^?{8w`z8GjuxSw(aDu_bV8{iAmw}O5Fj@!3++ch)nDBt@Yrv!z>{t(`>cOrC zFzo|-J^?e0V0I&z^MiexLB|%*`330O3f681-8(?fm!NkiSic+e?ExG2?w5{^f8T!2 z(JB2sQfmyNwEy@EFaajO1egF5U;<2l2`~XBzyz286JP>NfC-d50XjNdg~U}zT!l2o zRw40r)x2GG$!}N9laMDNPeM+Q;PePKA)Am*$R=bHvI*IQC2zt!j*dYQR5Yk)n5by* zLQn)RgzR!~u_1T)}d`T0Bsx{I4swUPf&_vC@G||0CYAJGlj3x?| zS+sHdGo_?6;88Iu&Q1YSbcSc&EP!7l9_FFp+lOc9EO=;3NGp*RBrTY7n@|;fYf%59 zZ;k2``qp%3YgT9IUze#*>f3U4n!Z)k$@E1-lwb+|2FN-*QPyR{qk$o)&CiW)0@t3=(HZ&I%QBpr**B_ zYS(}MB&a=#MaXoUpT1YnnS{wYryeBe;LxPV5(WK-z@``&I0^>iV5kiYC&0)lFxn2r z&VunIm^cr%cYw)GupjnEVpyOxI`3vZ}0oMKsx^ID= z-$3tM2PEs?27T{NfC(^x(kDPchk_0T9SS-Wba=ybHX)mkO~@u>6S4`}gls}KA)Am*$R=bH zj z`>ZBfpBvAz&2WyrQp!sM%$xyV&K((1r$QJDkaHg1v@tFe!+0#fn}=rvJ^cHt{+M#k zjC&0x9->}duw0WiA_%mu-|7SOR1bnXUS zd%)VgpnD(aIRJVOg7t?%-x09!80bF^Hk|+iC&A!pFmwhCe+5R)fzb<)b7B`M=NL-A z@S^0mX5}xy1egF5U;<2l2`~XBzyz286JP>NfC(@GCQ$MODCcnb5tkov`Oz3#e#Dzp z^Cs0LzezPuLY{;?2{|)@Gb7l9Y(h36n~+V&CS((qyb1HjIYJR!&Ck{RWz<@DAt-_u z!uYumiYVty=fs?Xrv+6>f$46AWflyTfx~p}-cu}JM=MCs8HQVP_fh11E|s8#!aU?- zI^IJ^M-J8*dVtPoIyyrR(0M}&Lk62gy*)^0g1W$NvNI$a<%s} z!@WBPpP++dlQK&RjC~8nyTC*@*q#QHm%)x6Fx3loWx(`LU{4mzTnDo`Fn1H|>jNEc zMJ1hYgRXbL+IK6S4`}gls}KA)Am*$R=bHvI*IQW2_1DNIG1WKYo_wQ?gJ5 z|C-e9jJy!!g-}FEN8#k0f~N*mh&2oHDlNBnR*smnGe?L5rsxdq)C?)zn>1OP!DImJ*aoJ8U{?#64uL(pz)Tp-wt~3`*tZ{a90Z+*LDvzm z_890s4th?2-jiVcY0!5DZ2SuJp97mNfPsr(@M|#i4H*6ojC>D9e*j~b!1xs~@gvxN z6--_OJAMXJzkppgVCbZOrJ=(yI!u5GFaajO1egF5U;<2l2`~XBzyz286JP>NU<@Qc zLx*dSxCV)9kjB^=B;Ku>cdIV>-Ku#K@+9O*$l(zj9>FGL6S4`}gls}KA)Bz|O_;~f zF)4ye`njavOt*y>f+Bb!jGqgkh=$HgX`N`I=3koVUS!Zj+cJrk)e;TsCAt;4>ruI9 ziri7K$)b(ZpE*F4+T%? z>5N@Z=buszGRmAhgjJ}`3t%pN=_J)PX)gPf#O`n#mo7({9R@fTnMOn?b60Vco%m;e)C0!)Aj zFaajO1egF5D0u?(bm-~O)1jwBPltC+XA`ms*@SFDHX)mkO~@u>6S4`}gls}K;TUVe zJf03$=8vD1`SdIl!M`TWI|DBSc_9?h)2Wc|i6+iItBKa7G?`h-3s$8=|F zR%hs6m#I(c+j4c9zE#xGoTU>m$`3V2CkHN0K1XL{X6a<4o@1+7dq_Guecqb?-TJuI zD1`kH%i+{ss2h7sxy}3@c{#E~e72p4u)%^GQ^Z)UmZ|wEm=jd3~*0)W=addEO(q%ac z=Hg&q8|XL*I!}YHGhppkp!*!?xd3`Eg7sg6zHh+B??C_eVABs^;1U?T0)~DB!&kw` zH8A=!82bf`-vAT8g6+4!NfC(@GCcp%k025#WOn?b6fzl^HM~ADBxC)7@kjB_5 zB;K-`x2!JtEvtDF@+9O*$O#ghAi*YN6S4`}gls}KA)Bz|O_;~gDN_U&^>a~wIqeo+ z2#Vl^Fn%tCB04%%9G_F@=%DFRVOl6mnlRmM=iL8%Zn1rxS%nEY8jKn1s67_ovp7U& z%ni}mQW`ABX-S-f5E;`o~UC?Q{TxOl3dpH%+!?91YbSpMF?n|v%dits12|8BW zB(>qtuml|(n-p2rfvyk0+7ChZN1*3p(7OSw{}l9X0vkUA{hxzPTfo2)x4DCkhop`b%Shc`@T6S4`}gls}KA)Am*$R=bHvI*IQY(h5S7;C~jf)1DDkDn#^ z6f6|Mzb2JC!3#lN2t^chDmgf3SVM!Pp)=r7F)G4N0V8wHx(u~dGmVS<{Vf$ez>9l6H6kXEkP3M<%T90j=GN`0e zS2tjaj)0`|w&C6>I(c_O=jjcbq|73ZP0B1kfZj`B{T0ynBiMKq^j`y;eg*@-fWaGJ z=vOd&3yk~*M&CLt8G9RyzXK-T1>2W^$)#Y&axk?5>{{>9l4($5? zbToiYAL#l7tZf9{8$pjB^lk?0n?PSP*cdo0B_02^!O|(9zQR2}ho=VaA(f~VWz?XAJM%Jkih5`hg!KT^Fh14R7`G3gIv=WHIz&be*n6x+`6NioOP{B}&*IUjW#EV`Yu_)2~|r|sP3 zCx^6i?7BuY2^x`N5jLaR`;4_h-!~Ea%H=+xihF&XNm9+}JSEi}9GFyCc7Xmb!KR&H zU^f`t1BUj3;eBA_02n<8#twt=BVghf*nS*Lo&Y;ef~nJB*BLPV71(nQ%v=Do7s1@u zVBa^OqYHF)gRV4Kdl_{1fSz8^n*r;80)1Jq@jB?wflW8TKpz;aiAgml#F_<~sQH)Fbru<5#RZ?GCC=C6NS$IBhoX*{Qie>9) z1?f4%uy1Cx2TWl+X?Dqt0ITg z7}T^ue_~XdXUJEK(j76$tx0Z;`U|FizhJuaf=MmYUoflZC+go}RxSE>m{m=Gp-e5& z{G5Q4b8M|AVCXnv7mIG_JZOGSO)9JoYUnh3%+`Ah9iz>xw*7yF8#*{JX|lWnhTa9k zOTfrdFuEL!tpMXI!Ne-C{XH=GKG?AqOsxaEJ^<4nf;}IBnUBHj1~B(2*tZFEYzCc8 zpsN|I4S?=#peG1=Tfq7d=-UN0hCzQT*c1T+`@vuo3>^Z)F)(rzjK*Q;#M)@+7-;A) z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oYmm4GiEEI?*cv3>qnh`qF8MvG zc@pv@l_MtX%n`-{2IxGDabsR6hW)6*w+urID$1Lw=RCZgGcp5o zBGSyUwS9O*>N($R|E1`9PWPq!19TcTFB*J+PU~9py_0dwwsN((#Zb7PgCmn3%SkYH z8jPO-6JLSt=fLCzu;U__`Wo!|226hk_IwX!egLzVz}ywE??=#)0i8dAt}Ix49dzeF z&rQ(V2iDgdk@Pvh#zmmt2{tVT11>PQ3=GwR;W{wl2BWLNm@PgeiORalxz~1NWhB??fU4QRX zy!R>V#J^2@^R;QuSHJvPb+uiYJ@1uy^IxB*K1k7N%^n_H(Wz-1py=T5PiVluU zsw^LY$&bN~4Pfe1uxk^T{tWE-9L#J1vtNL@tzh4F(9r@qL!fIHSQ`f2t)M3YdiR6% zQP6h?Y>a{aqhM1U477p41Q1Aks5wfup=>XWX4a@|= zY|Bw8>EuF3IY)NfC(@GCcp%kKvyx-I;kI z$P1x}l1>%86HT!S)Ny+2s^zWQo zY++|sL59u)J(>skHN)Ni3Va-E|6>-OQ_mT5dd`-i^qi5Ip_7qjj;+RZOzJt#8{wks zIW3pVtW$Igrvf_ABulqqlYOPuEIs|y@bnz3)N|ULro#0c9GTQuc7wS+VBcQQu^)6s zLDwO$HU_$nf}S|&Z3F8Qpzjpe*be&7f=x*oB|`?g3&H8)(ytfVB#{^ z-UB9k!Hx`=`U&jHg6ZpEPY%r71hai$uI89zp96Hf13KS5CiNWGl4I0!O271?#vn@j zkG}vDU;<2l2`~XBzyz286JP>NfC(@GCcp%kK*S7=7`NfrGZ^$9tNrrc(_vo)(T^smd*C-rT) zI!)gy>STRuQ6JN{6Xe&5oKa&?(+2&CQEkR|8f`|^B)2BHHR>;z{{4dK&I=~BOn&~+TF zJpsB;f}YdxbG&Eh=NRbcFaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x(kDPahkg$I z9Qrx*b9k$CHX)mkO~@u>6S4`}gls}KA)Am*$R=bHjbM*sP#R(fQK-ZqD6%=8lY~Qz1+RxH*#+icx)3;o~?vXUw@dn@i*7 zxTTh3YiNv1H|J1WPto0+#aHsXIc?`IKRKkEW7mi7Ov2C|#UiAe)9^p61NIr!&gZR< z>F@g!-eOLml zA52gwmad}>)6bbTD(1{9_a6JKq1>c}!uarySMWndRYQ^=CruT0_(d)_Pa=D$8qebDTjnp9XF)X!=5nD2LS zn*Zm$`!Kag{vP@L92}YSST=%wKiISx3^akkW-t^0!`r|}5RA5fu@D&F1t!8^dn=fX zfF1k6R21ww1g2wP&rvWF2eWNpE&=wP0v%^S=U1TX99Vk+bYBELUxVIn!20h%-}hkS z51{`N*mMO9{0IiG!p{j^qo2dsIZS{FFaajO1egF5U;<2l2`~XBzyz286JP>NU<@Qc zKZkw}{T%u^^mBNhbT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6OOSa%;V>9MgI6%kx#!u z5&Ub?xHIxXkQYJ`{T$`L*tZ$3Yg8fqrG7c*{wI|qrt8cRh63j2JkS)X95HjIvKY$E zaHX9pDmqrhVz8Oj^An}^W0X%Ua%+-XO@E$PYwmxKe6O|4Zjw5UwNf{0CS&4^%@UAyj;%FvTq-)QjrB!WbhtHG=X%&h_YyrAO)(D@;|KIj!Q=;w)Z$^=x~M(6JP>NfC(@GCcp%k025#WOn?b6 z0Vco%m;e(feFAiJxC)7@khltIjIBcAt*Uve>XP58nkOMoLY{=29Kp#EY(h36n~+V& zCS((`2}|CDc^n;+BDkoZi~7y9TX-QTf)~R0xe$uz=*;BYoPuWt&6G2P77D}dhn&lG zzov8dp1DO;cI3wupy?Deq(&b@V_qnV@R)@kQqYGM@oj%lM`yCsValZzGpb{B{wY7y zXe(FCf3U4n!Z)k(NuH-Lygh7JwB({icUh>IkwIYC^|=fJ63c>r}Y72bb4>DQ>}m1 z=lQ5|p3l>dS7g5K?5{SMIgCD^zV^zR0n z_JD!CU~nH8Isk?bf|0{u^avO`2F8zri4$P^NicaD>^K9az5=_>LD5NHprT`-qQeB3 z025#WOn?b60Vco%m;e)C0!)AjFaajO1WKO(6&)@@;vyt2LKka)Lh-mSXicdO<} z$diyKA%{nBcm$h}O~@u>6S4`}glxi+H(?${r%VxC)6X^i<&;}^A;>8llv~Elg-}FA zrwZ1ICR$cYG_05CRy5J}sN6F}?wG2H#WOToJi5U&T^dYs8jOayG5JNE+xzDhTi2OY zke~A~n$4Z>Rp9Da3V+PRo2lrGIYnp7P>RkQ(i<{JPvy3%GeKQoH(G@L-z@hV?O>Ts zrGKxv|2^`()-tM+(y-He$mpW1Dfl(chfy_-)+Md$NN-xXcaX}MfxouYqU1wL<* zrCYJd%2I2Vo_=b0ijKu*Rh#3(QFL%$8tl6aI(`J5S3%b` zu=Z!r{R`;10eXK0>u-U+-@wMVPDuLS2Akdi1Mh;tC17YN7+wxWR)Eo!U~Cl_e-BK& z54Nublk32a55UxiVAn@r`eU$X1DN>~%x*d%6`kB?Cpbl?^vf?wev4NA0!)AjFaajO z1egF5U;<2l2`~XBzyz286JP=*Pk@RJ6&)%%RCK85@Q&$hLN+0rkWI)YWD~Lp*@SFD zHX)mkO~@u3V@;Sx(cz+e+9}ku1nm^L@_yXdDg0~7zh)5?ok~v58P?RG`<0Z6QBigZ zn4vQax26yaXAb-u>mZN0c$I?Am=koGhZ1x~W`<5ix;eI*Um@t!p1DY1S6-x=ov8f6&ODUCN6;O7s2G$V8=IL>N~LO zdocY2*mDWYTmiE`g1M_;-!;&Y1)bL+=(utebT~tY2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?cDfdnY%a0wEZAaM!O7+ZqGTUGN`)g`}GHBUmGgggm3If9cT*o16CHX)mk zO~@u>6PCOQ^9VX-MbOWnpJAk*!3#kVyb#9Eg-}F6heLA;8yHk6I$s)GN0~b^qE3Yn z1;{yri8O;aHU+*k9V*VG0gnpop`SD6{G820`8gvqK*u4~99!Mtg!FTMjJ)@MS|8V^ z-8>?v-8?iQqz4_X-=c`|St-9K-%%1nk zy!o%sQx{*!@8`6gyZq$v{2U?uoSMjeevVmf_A%RIFrvp|nIzxXdCDX`7I|pW zW4Q&oegkXYN=Ulj20ib9-gm+JC7^F9*ti_@uK=4?f`L_F@I5f}J{VpLM%IDR55U-m zVEiL6@iExG0Ze`hc5DJupMhPUgXt|`&lg~3E12C5=5~O6UxJQZpfe1*TEW@~=-v-{ zq6z8ecn>8wK!*c#m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz2;=@X!zLqCUp4*eYZ zIlNgqn~+V&CS((`3E6~fLN+0rkWI)YWD~Lp$5<2Q@pHH$fBdY-r(dB6{xxabnRp?{ z3!#X9&UERWXyQ<{CYoQ>#F_<~sQH(4b`}{l(Y8#YWwk`ZdWmjD?s`=2nId;g)x_c% zn&^E-6K9{*MC)_ff59N0DtYV+h3J3G!t=olm0~G7T0xG^K!Ij}PgBG%7eLJs^0ylN z@QeHE0z_@rgt>?ZQs9rO5|gauQp6FeqHhiAU-Yd}eL~-w?rhEK4E^ge^+|nOu1?dp ziaJ@}TGYq&?F9L?VlmhZYTBSbF{;h@PNU7Jn&j3bw?_R1)4yLZ-Fd;Jmgz5;)$;6hWAfOL8qN zfC(@GCcp%k025#WOn?b6fs!XcKZh%jxB`hQkjB^wB;KZ)x2Z1qZK`<^@+9O*$f*&W z8o?%H6S4`}gls}KA)Bz|O_;~eDOUs+^m9SKfo2OY1V!*d7(W+65&azHzu31Ku4_~w z{iS|6=l&;^L-{JdTi2N*Oa;u)`Hwhs|L-;ud#Z4}(v}(3N7WzG&v{HbH03tS9ZzRm z`Z+$S=GdCQgP-$-_dwD8obF5c2k10xUNrasonD{WYS;aoN$_(N>E{^L-oM}L;tVe6 z;LxPVvKI8O1Dieo10RCHkHFB!V0Z%<`4o(90%M78KDZZNY4%>dKtYEIFaajO1egF5U;<2l2`~XBzyz286JP>NfC-d70SY=4bSUUh(4nBi zo29b}*@SFDHX)mkO~@u>6S4`}gls}KA)9cFHDMk>hfDIu&ysu!7K-3slgeEgF9dlZ z6j9Kb$)Py~4-A?q2L>$^rrQsBA`Wi7%q^;~BR{TiM@LSXD(>)_L4MjyeXb2PopRgI zV{~qh&lzP+=b!qVopL#N2a|cq)j!Lv!KSEh=v#}rK;KSK7uby!q5n6_*Nt|tOsCSn z*WCXe`Ce<8-6WM4Yo%_#Ooshpvjn7}V{6@dN@_Z8=Y>(zbXv2A2iJ7k)|>BDa*T3j zjvNto@6N$J9UPnVS-uBDKY-y&VB`uI{Sk~^1>@Ji#Lr;+FJSTp*zqfvx&?Oq2BzOS zCE4>fn0W`xz6<7-fPG6rM=j{A16^*gb~WhsfSxs=*9+FK2YvNmV*}{-flZ%)fkrU6 z5e)gk@MbX51V)=rNlzyhIK@FarC)?rwxDtsgk+>3RjIBiC{i=Dt>XP5DnkOMoLY{;iAHnevY(h36n~+V&CS((` z2}|CDc|0AXBDk=h3;P8P7hVX8;Ds=LE`%a_Iu+7A(ZtzjHPQNn@H_x;XtA(QNR_hC(~nurf1f}9A9iK-yjSMUe|?@>S2w`XG0RArPYlb^!Ldn~Wjh$# z0mi=s6Fb57-C%ML*s&K(?E||Gfa!x^&tWig1k4@-bH~BH6QJW1=xhgFXTjPe=spj6 zIzVqHSf2uY--3-@puZbzN`ryRV6X=a^@8CH82Jf|X2IBXFrEVwH^KHkI6BFi({yy` z=r92$zyz286JP>NfC(@GCcp%k025#WOn?b6fzl^HM~99M9UVG4baZ&%bT%QIkWI)Y zWD~Lp*@SFDHX)mkO~@u>6OOSa%;V^ARsQ%{l~2b)&WE6EMA?WJg1iul=;%~IJJCeT zYKeyR65Wa>+8&jArpO&rHL-YxMvX^Tn5IjGX`zr2<=1rjm7Tf8_H||zB(R0FO;cUhH`X9W{yrm>N&Q~ z<8X9NU0C*iSs&NEnMdHwOtAj{?41vM6K8(rNB$?Hu*r*)ER(Ri(6p4c+iu#lo0k5` zwN;vJuO+=+n{w?n*QU2zdQFq;rI+q*_qGuN95BQv0vs^JC<-`Wh!G+fV~7z7IKl)+ zC}6;V5rPmDf(hYwxw)_Q0ecJ~!#%8BabpW5!U`rty5SZM^cbaagx!Y-zVuP-MxD z`3LVc*3m&%%2ichWE~y6H|ny?0z2k_oiBp1d0^Lku;(SPcOjT~8SGmG_AdbkUICre zVEGELq6Vz=fNn2XwF<1R1#8xT-a4>$Jy=%{*84!eA8dFFY-#|T8^K@`*s>jLZ3f!{ zU?d2(?*cnoz|IgDYX!UBJ4hX!o;L32Sn@yrBJUSzWdZB}JHQUG1MC1hzz(nj>;OB! z4zL6406V}AumgE_fIB)o3yEhT@hqfKHVcU_SIw8J&im!6c_ZYFkT*hJJ%U${;3VWE z9ii}+*%5Ns(V4(&=NMWyh)Oz| z+qj$3*~wxR9b@gA=h3ezHFLA$$GM?HtK$^PUsYDgDUzos_r>x(%6*AENx8SmH**yo z-}M?gH)a)`7?pEG{MB}9=xpAzD(4NIcu(erPGH}r@wyG2z%tD$I;FxU$2$x)bnxD& z$+92p`2g%a2qq4JeMi9lqu{`C&>02GV_?N;u(AtucY{?uVD)*hrWf?a!P*2^_X$|v z2l_9A4gFx#6|i{#3_9A0El#j?HrQ4UM&^R;6=25#u(J}3xxg+r*s~bytpXEE!M^Ht zYUuQ@Xy;XQxS_)iumkJ>JHQUG1MC1hzz(nj>;OB!4zL6406V}A(3 z;f4-hF`bi;laP~;laP~;laP~;laP~;laP~;laP~elqF$?h7Qlkzjfy1bHll8HPsx=w@=)drp`F2Ad7a6`KpcW1&0s?SYzl(SyTD)z*b)L; zTfw&Xz(^a|9tJxiVCMla)(&1Z!0jBKf5h{Tc>d8Sn}5WYspiX6 z=lwF(ybJ`FT2jG51<{N3cq~ zBiuSWLJr$Glc{xLmqIh_lJ9A|wDftqthk6^0lRO#>mf?6_iTK|3&n8K^a)A(3*O*DD@h8 zt*yAMuypPe+XP#IB$eGwUzUkdC-rlLtLYGxbS`-g+dK0lGf{tNOs|%V14E=)D5g4uExz zL&SO~=${QXl!Hxk!R87uxBzUa1Y2ETn;VQQ2HUH^j-_B{H5gj~cGZAA9;OB!4zL6406V}AumkKs-W}kQ4wrPeq{Ag0F6r>)(m4q^2{{Ql2{{Ql2{{Ql z2{{Ql2{{Ql2{{QzSrTR_>F~7tTW4B6mn^Ij|7&vXu7Gz0+7WVC(y{&&mu`#}j>b`m zDb>(P|6;-o>+DRIq%4-vxqheSx}}?pFL(k?94Y-qs%~6E=LvLwO44uB@!*Ymrz8Ez zQ>C#~YAF$tY*p?p^4FF70{N@Tec|AJkvv8DzF59Txi66?Dfd?SZsopo@O}*4Td4+P zk$Wu47X`8pKUp9OA}lI_Y9#qxC< zI@g*xxo+sVshlG`a}QHP=kS8_IdAB6pUb?Aj%WRB{bh6E#SX!DY=-{nUk!3qr-3-F8;7>j{jr@@{su(umb^niWm!Tw%wAPzb&g5{rr z6_>!u&p`L*VAWNy`U|jT=3%0D7FavyFcoy_UOdbN9WLmw1MC1hzz(nj>;OB!4zL64 z06V}AumkJ>JHQUG1NnD=3pzXniKig(6r@o$1&J?K%@?cA`^BnxBjk;cH$q-Jg4d4V zB;+LIB;+LIB;+LIB+PpfW+>q)j;nvv^a#+y0pI6Swdexx& zY1N=qPlYliy`!UkOhEn$FhCv;NR_m(qB-ofrCDYHzIU5{!WeYngr9gj{{UON%5z1Mcz1CxIE+@yzRJo(&=8D-YTo_Xq-XTSf9 zJW@?3(6>jwrW4+*sp+5$MSth8Mvi_>2XBq)EEQng0OTm_E zuyqC4Rs%*nV7nLWSOs?0g0VGVR~^{19_+0L6F#ud5B9$W4m5zytzh{!u;Oj7atG+% z30A!WR__LD-UYpTz}mfF-9E7XebB!jZ1?~*ou-3a)3M}#3P#>9(#itZ0d{~LU7TytPN62AKXB<^e?9$-}?NZ<& zyR_yJyX1P*E?s`iE=3==OQ9Kd$@jFKt321LJXR^Zz>=}O96HI*wmS+AsXGb=j&0;P&mqEvATsG6g#=-ilZ(urSLF5AALRD5JqeD1UredpO{zdO|JiF=EN*KrC&k?dNgSv5z$ql5QGU6#XO zQwP|53=DRHEm5#F2DY6BBVAy7H`vhwcAf`gy`8#VpMZ%zuw zbXs3O!b|CJM~59?2iO62fE{25*a3Ec9bgC80d{~LUcGf)u)QAa@PVCvF!mPM)d2Q1g1t>(VmsK^4E6`WfgtGI4VJ$P zR_p;Q_k!+yVAcCz^?tDC1JHXAtUUzQ9Rcf)g8t)R!wInIB-nfk41NfJHQUG1MC1hzz(nj>;OB!4zL6406V}Ai~=m3OYO`|JIq3&jkys z#Q&OHxs!NDpdBHH1)T}JaE_rxgQ%gSxsAIig`F&x(9w2t#@+Y^&E%Wj(4j&QmvoA4 z)=~>C1as^*S}X@E!j#f|A>G@RCyHg?^-4NFP*&L~m9MR)Ge&;Cte{j<{x70Gxu6Uz zR%lHBM_7mvjRAHjURU z>9|~)Rdh;3sT}qe3}0GDe-$0PH!8Dy0(SI)otME_KiG8z>=^)i9Ua7k6YQG}_LqYL zb3x~Pu>2*kVj)=hGU#3eRxJUmUjb{Df!^g{?W+A?QEa~vF zIoT~4G(n0!u4(6JzJ21xly$~QmMjW7daKzyfhJ99Zg11^2kQN$H{w}t=Ulg)v;KPR zoNKL|T(@&#RLl|a%A?fIdHao1IdA98y^y(`6Fz$W9%JpCQel&QXAHG-@ZPA!5(Im8 zfxRtYA_Vreg8lD-18tylKUn?&SaA@nJOsLrfK^Aq>f>O|3DA2ItUU$ReF)Z{0sUve zhI3%k1+e)eF!(XpauIC(6l}W$Mm_`EKLJHQUG1MC1hzz(nj>;OB!4zL6HcYxbD+|J>44!3i- zox_()=OpAL{4ikUGhC`mzF+nmmEK)#?EYuT?#KCG%hB1RuH=XqN&H*?$U_;+abC{RPRAx=q8^+7)%wkL0SbW6!FgW1xU@Kgu;W z^EWI05H9JImWV?6tF$Uj2`ytYMY%7Q?@{hc%esg+`Xp!F!`Jiwo>`g9D2}=PO|OGO%JfSotdGUI|vc23Efg)~p7- zZ-BLH!Mb%|{hOeF1K6+;Y}y1iZw7-~z?Q9G>o&0MZ7{L}Y~KlXyaRUb24nAnU389VqEk9OIIXCI1sJ@_v0*7Qha$1MC1hzz(nj>;OB! z4zL6406V}AumkJ>JCJt=xTM3=ka!vrPeU4I(~$T|)qJJuykDuBH$vVBc_ZY7BY5Em zPC`yXPC`yXPC`yXPQtt=VTO`UiB;lR{XDDR!u1y35v&sL2)E9Tki(LW^{2RWW3+HI zj!I0ahEDny6K+^%XS!67#WFf(N;hL)kPP3Z;zzil^OM)mxiQP=IH{Z?TZZDlX<-RdC%Oho$X|n54&7%*OS*GLB$;xXd;;q7r@$&z`BpY`ir3dQ?TI@ z*z_6L{5crB3buR!w$408Y?}o}=78-lf*td~&iP>MC9rED*z+>jy9i7y0sCG7`4BhZ{QF(BVs_a}shAauRY9auRY9auRY9auRY9auRY9auSZR zB+Ssz;W_!Y&YXO1SXd?g*W}J!G4BYpBjm85b3ZSgll8Je_tUaLFGvN~Zq$2BFX^Zs zc_OEU9s29CDCrn(*<>u{OhffX%I5YIUY47Z&NTcvS*-;06ke8_cbfg5Lv&kt>;PN@6bQzr=k;np3LJRL;Wu6lGYjkfBR{6%1bllX>5uVRb(uwxe-L#TU zpl^?UNykxJWGhqpIAhVrv6f1-NKv$Gct@uc9i7IJIy!i3)Ma@Stlj|DYy`cVz}n4V z-4?KZE9l<_HoOfs?Ess1g28vdmfc|MyI|WMFtQhH-v@TQ4|eVcV;_KB2f?00VDAwy zaTM%34)&h_2Tp>{(_ncQSkVnu_JHp5U{xZ-l&d1g{;zNytgaNytgaNytgaNtpK}%+S#(uu43spC|Q8+-~6=!7A~NaO>;{ zIqc|+quz;KI{ctr3Or<&);waDT#wqN%a7Tm=;L-NG{Y|Wp0;z3=UR`)DuowVQY{_z z7qrfex}TH&waLcL1WC%GqLWpxCY$b!`Ze@dtKX^S>j`+{SygLt__C=#nJt~WlvYov ze7ABxhVHFYn6bz`7Uhcq**ArrDWLvHA>9|!eSz{sVe*N>!6yo-bAul(l8@b{{6vvl zs{BNeY*(Hrmi-0vfQ4E)KTuwf-qOM27OJIOYwL`WpD!yYm6ZRB=<$Lwuvnor^=MJ* zHS}6raakcXUThPT^2^;Qzlc&V6?BB}-Q(2K+1Vb&(mNmW5T~e@*V)74eQhJ3%ZmV4i-$9Rzrw1n5rMXk=T;x$oOp~d^MD-QBZfk`~EKlURu`?~Jiq7B)KbjUz z4!?a0@2ejz-lK`?R%Y(D~a90fa%gRv7}*GaJF6xjPAm^cIWodx^PfddynXB;d~fEAyB zm3^T5GFa6QR$l>Y20*W)lUVBn>t=)X<)D8q*iZpBEdZM;OB!4zL6406V}AumkJ>JHQUG1MC1hzz*c!0WRq96eOO4#8Z$)*%TzcS~Xv- zI`3Dj=8cdyLf!~@@d#c#f|HPwkdu&;kdu&;kdrX)NtmIaW3fs+qn~H=7jm_QcLb}% zJHoBABjm84Gl7@RF|=%udV!g1Uyr*fg`F%G(HZK_q)Ie5^oyz7(2>7Ni_v7$(7Cp9 z&dpgt$CsamPLRquB76!BomusXoHumhJ((Lifqk3C>o#=a-lE}UoDy1G$VW>T4Xf>x z(uzWX#)9Dwjum6&=z&6Nve0{@Cd(^e>oTxyIT(2rY+ng>yasl@4#rl4U2lLrYr)=i zVB$@%Zv)uB5ggbAIvc?9MzEp@tlSQ|o5898SRDjwc7fg&ur>tNwSx8Uf&MnIAq+M} zz~%#BupMkU47PTFZO6b!C)gfEL#HFg4INAVXJ6#~`m8K~9bgC80d{~LU70a|gq(z&gq(z&gq(z&gq(z&gq(z& zgq(zWq^LvuNj(ydd4w+f@8@ynfE8Yv-)bPdg_<#T*gcc!JtFrs6z}}C+#6__0Q?UOM zIPe+hyaJXFfEA7t#7ZaVo()!&gVl4vnhMan0IaP9>s(;H8}u&*8>+yjrC@V47+e9i z)PSuXu+0lbR)OucV8;OB!4zL6406V}A zumkJ>JHQUG1MC1hkbeibox}5wc>WR3KN@B8kN7gxe3|OJU#6NjLf!~@Bji;hc-06_ zLQXdcY>QnAFCa87CU{m5x~=rk9rVa}dSIemnmffV#lK~j4nJs@0uR}L z3fAG7O7HyzsW5e$hOZ@^TdoQ_c7qL_X*Z=i^VChR=TISND*n>3H~BVI8mp|6Bgt0f z-Xecpxi8@Lb0n$kZYtK4iP9LAb42_LG<0^)@#egtGt&Awj##s-+tBe(7)yX%pMX7mVDDux z(GT`rK|`m1fEzlzehxdp4zL6406V}AumkJ>JHQUG1MC1hzz(nj>;OA33LM~u4mWhT zp~DRwZs_o}(m4q^2{{Ql2{{Ql2{{Ql2{{Ql2{{Ql2{{QzSrTSw=uW&22W;?yO{>7>S}?cJHQUG1MC1hzz(nj>_FZf;F1ncL*i*jJPm1-O+(_VRrA%V^M18z-UxXk zS1AnT9`yOFE?` zqHu6Uog#S(^$X}y-i?9xsb8=nMaZ*1=xK^B`lFp0A_usUV zPM~j(eo4o(NK?|0sHEdKY0;K+Y_jX5W_g{l^vdXgLMpS+d!sT-7_5kZl?Om~J6LrX ztnL77j)C4zur>;OB!4zL6406V}AumkJ>JHQU)-vKV^a7l+tI$YA>k`7-ros*E0kdu&; zkdu&;kdu&;kdu&;kdu&;kdttfC1Hk=4o}O!b*ANW$-*k}zb4o2EW9Joj*!EW&NwQb z*rme{+NHolc4^HccFFaqUAp|3U5Y+#mqIh_lJ99d7kRD~d8|@+!C*%xRo1!o#dPV$ zR(2*x1zB`-zD)CGDtb1OiQAXy=qQWfWYs7dVf~*k^G?k)lgK5ix0DD;wkr1)`RmGk z!C*IMbaiz6*IP*E+5$Sc?&!Fwo+CUx=;(BZ&gQ(M6BuD3o%ogIvh5qnias(e?sMOr zwxaJm`|NkitdBkO)HBb1{~0;3tjJcTJU^D6FSC|P)PV9#9A43}37Z@)*DS7+s_5XY zQI(|ztoDF4UeLP=tgQv>)`0bOpnpBsP!Bfwz-B)fd<$%809zZuwk9yL9c*s~I|5*5 z5RB~tyIR1W5ZK!aCf)=4+Q9xWI1mAy2f^|~V8s!z@+jy&4pyB2t51S8r$FzAVC@-H zbn4D>MaPo==@)swNGl6q2iO62fE{25*a3Ec9bgC80d{~LUsloJuRA841H2oaSw@>7{t}`vGevX-%&EU_{dSmnkv%lSs7n^C?e3_q}n!9Cy zvM=qJujt&RRC;dCGCCWsSJC+aiLr=WS9HA8&Jn)G7*%xkyfyQWYC2k$jj(J2r`vg5l(Dmr*;RAo62*7bt*anPRt8$JP>`oQMPV6Y!-xdOHh zfNhQ#G2#T;pm!Zu`zBbo0j%E$`Zs|Mn`2bbY1$IwRdl$b!w#?m>;OB!4zL6406V}A zumkJ>JHQUG1MC1hzz*c!0j}t9MTaXoT+!i*4qq{ylaP~;laP~;laP~;laP~;laP~; zlaP~;lW>$JVTOthPs-?j&u#wijda&(Pl_=Eb3=-|Ck zlcfo4+7321gTVmU5(Ha!fo&~dBm}m%f*tRHoo!$&40c7po&#WSJD4~O_H}^$$H0M3 z(0LLpKLu8N2v(i}-Dkn7b71uau;wGs`!QI15v=f&0m1QnWu;? zvrbV%r*+OLZs_EH_C?;W(8>bX0d{~LU1Rr5y38zFCmymSOF9l=S+NytgaNytgaNytf<_aw~F z&?&M?x757!TV@Rm?u>9}g!fHav4eMnFSjG)u%W}N=47{G&;%*^czTOP{m2tHrl~Vd zlCo&$=rw6{N;X+OH5px;63uNK9^rycX^AM5zpAW|QzTDO?u+Gnl=~8Sl5%gA?^f<_ z&I&r~uUF8yF)Qc;OB!4zL6406V}AumkJ>JHQUG1MC1hzz*c!0WRopL5B-ET+rcy z4qq;vlaP~;laP~;laP~;laP~;laP~;laP~;lW>$JVTOVZ&wnrYa`WH0I$)L9I_|d> z@{TY%cZ3`kbS6{r#4d$q*d^c7c4_JJcFFN$D(%d+*ro6SLgQkBX9c0#N)O#ZkBp}W zCfcRBQ|wawTXyO2gLWzKko~7%DW0kH-d~WU)NLBR)?2tHSDhVuRwW&6gGN)R$>F!p zqnlG|mrA*zqcm?uS3@UHOXx(XoFk&%)6~#8e<_snhR#S!=s04{vTj4ivtF}=PAM8X zjRi)Q(7}77CQCcmaTx6E0At6%u1>Hg3iigp#A&dv3+(R(2YNu~1+e@hu;OE|@*?Q| z6s)=gR(}T8d=7fAg0)|Obu&*B>t})fIbg$!VADLXc|I6?32a#iw!RFuEdnD;!1h-mlNf0@wj|fE{25*a3Ec9bgC80d{~LU6IY^^y4iaCfny*xy_bXNNM#vi>Z-l&X1TP%HNytgaNytga zNytgaNtpK}%+Sy&;WcNh61NO2qqAkeQ5vIij)=$6(Ajq?n)8NE_qoh#=y=x8 z)?Y&>?oDgx*kqrh&`3iE?~R%)tH7RGuy+lZr~~`fgZ=g3fDd$T0?Rjp6uE_VISD^KG?h;41NH%90Xerfo(^?$WgHUIM{Il z>^up^PJvw?f<0%z-m_rh9N2dO4W0gvxS_*q=&%Fq06V}AumkJ>JHQUG1MC1hzz(nj z>;OB!4zL5GzyWUPa6^Y1I^59Vh7MmZos*E0kdu&;kdu&;kdu&;kdu&;kdu&;kdttf zC1Hk!4wqB7oWd=wVs2?!CEgKksU0DQ4W0XW-JGn~47#7z40=H-ymq5lNiXTBA9*6D zg&q3qvM=daZ(?`me#yow>5S%;bbdtQ3Pg!qO!tL!|24X|2&?=9<-SyYUb!D5KVMc* zDk=XL(enjmV6j4L>d~UqYv{GM;*)mg_UQL?!YedA9h(>{yLKDu>EOLlpCtkIe*zBlfzHps^3TDFt6=39pnK+r#Hv|f z^&GI~MbJABtep?my#&@T1pP0A4U530C1CR_U~n1OvK(xE6>M7xMqUHkUk5u@gPm`H zv9(~=IF`QA>;OB!4zL6406V}A zumkJ>JHQUG1MC1hzz(nj>_Gk<;GPc8MBO5jHB*} zT{`@rT?#y8m)1ODmt2q9rOS`mrRd{!DKx__`JT3Om*-lS$0~&vSW-0&IHMlMNKC|V`e&PG-Gej?FszJ8UE56`ryMSKflC8?UMgF>S zUoa}y((xB4-J24ni<7CNbFG_`>yD0->N&#oQWte}7H)GrV!KNz@Z3%Xp8LkwERU2~ zrpc~HHIF`AW|@vhCo8X=h}Yh0yX}EVKYVV|!!w?IZpMr<>toM6^~|&1e?|_Bu$E5z z%5vHE4dv%${QRA^qVGKW?01KL(B;x}bR_ENII1k#wRFbH(QTUI&RBY7^gtnXS?Ik{ zmt`kd@eWwI8+5-5R_y_+_kuP1K=1ot?S8QC1F-%e=syHD908k-g3ZUl;0dthB-nZi zZ2J(5oB`X#7Ae2~u>f{}9bgC80d{~LU>@_%3RY)Rd$}pbzf&%Rt25w7jY(|QX}Q?HV%K<^*cI~bKR;Lcf(poBXxA> z@t>@k&W7vNbdpQyT)WTJQaWyG=m^hI)O6ytYjR%GX*^$S8?Ur+#$oB6v8CNsL8&D> z<{!NGx;33LgefOX5j z`sJYiRj^?t*z_9M{5lw14Ys@iwyp)+)`5{X!S)Sc$40Po6ByeJc5MNBwt~Iez{J~N z-wv>UCphp9=xhPYLtsTKSot34ZUd{rV08qnIRJXw!P>)U>C|;_OUJ@39d>{nUQ1*`dj)p@^QHE)Ex z5%NaJ>qqeV5uAjagq(z&gq(z&gq(zVPr?i>9Scu_|8kSyx!c3to-fgz;LV0No13%Q zI_c#X361`Zv#`VN+n{X2*|TzZ;a>*HBj`w|`(Zf13G% z&Rxp7JEij7%KaF+x0YIjMeeaEUlho`DfCPMRYwZxzL4$lJiv%sM(=>gEXF+A~zp+3ybIyr2{B$z0G0?AtV6x1bYP zR%9Di&?yi_a^QOzyEqviq#_HwH7c?k2kTCN^(R69DX`%~u;~ogd=?Cz16wYDtsjAH zAA^yLVEdJHQUG1MC1hzz(nj>_FZf;DQbpbhx0y1syKv@FmkZ2{{Ql2{{Ql2{{Ql2{{Ql z2{{Ql2{{Ql2}fBHW+>=N#J*>N%yQMWRIZY_?=9;$(b~S}gS5sKv4lY_{;XK&X3pU5WU;=FU1nr#GK5pmm z@;U4PJHQUG1MC1hzz(nj>;OB!4zL6406V}AumkMCC~$z=IXwS}=O6L>qfs{hh_6!3 zSE{56Ep>Z+6vx3lVrHAgIN5<0w z6YbL6DRwFTExUC1LAw-q$o^BX49`@0@9OntreXaU?P@yeM{-rzp@nL)Xy_Oy*Ic)W zGY;*WlIz|k<278;DJ>C&@>gjUn-W@?1?%9Hh+_F3<-SCoq}*EvSI!wFE9m@yL|H_s zd~GG2G4k_e1*MYme-S-iPzDw&w5A>{O1*|&Yb!1*q{54Bg3^7t8{HRC8l--X2=DHu zlFp)@k8)np8EFL_N32=aFX{M;hF5P&=&d{Y4V858-l)v-8QA(c*mf0+d;zx4>?U^1 z0z2n`u@}Lvd0@|cu=gb}u@LNg8SGyK4lDtkOTqGLuwn&RSp&K~V3ilFUIo_Fg5EV? zZ5>#*9;~ki{XVe44>r98HaCF5MzEy`Y~2pFHG`1=*dFYrl1|62ZeBu%m(XDc*a3Ec z9bgC80d{~LU-?81L)OT*D`p$i0Y?epLEYoDyqnbw_F0)L>qmz}_PQ+{PwcYl> zq#r&v>ERhqJ~v}Vnf0+}o_gll?>{4VpUb?Aj%WRB{bh6~12J7dLI zIeMTlq4!2jmUqF9Jz(cvFt!iudLQiB5B7clCJusqhrs?L;J{JP*$I|M!HO7I zc^Y(gfmPjLbq`o`9`yEtwQ;a60oH#4`uo6!%V1MK*n9;H4uCC=v&2>>*ftxCl!NVa z!Hx>Ba{(Bu1iM^ksiD*3KFbZA{Lj8{Sfu>^#{$>^c7PpV2iO62fE{25*a3Ec9bgC8 z0d{~LUahQ!-mfNymU_1%Ld&~%Lct5 zN!M<4aMd()G~Ygv)4~q@by<{jGW2NDn>FSixgXse+qIh&UvN)n^!9XqL@VYLh!VM& z?hEPuYsv~bR{6%Pq!Xixj)Z2YU{Ky&YiU7}(c| zo=$(1dpZ{G>97Or06V}AumkJ>JHQUG1MC1hzz(nj>;OB!4zL6HcYu33+|%Kn4)=7p zr^8oG=OpAL%(wax?lIu~sbonv66n)$-g=W|#-_v&P@?7ikSf%g+ zORA}({({!J(bRM_-*>mlBX!&7t8l3 z_a*Wq<=!gat=yLm-jAVsYpF$8ApaDqA>YH;ouX6)VRTq z7RkqMQ+}dIE>(V_NVY3a6wCesdcZ=R9CUFqb#!jbS~>};=Lm`N-+ZTf!Rbb~*FjftAtpIy!z+MlS@Pd7-!2VirU=8S82bRCtLmi!p4L!V;PX6a# zeZ-l%N@E#!VDdqVs1`wbAp=_H?29r ze-SU1N2xGPrV7&wQlVWc zRJU`|zkVXueVu996?E)3w=whlOHjxiu{<)3&ztdiGd^z?9X`!w__QNj45~n~ zJUs0$ADjNub{->#A8;moi3<;;kLkzJQkKb)Mf%c_<1&p(M|v>Jq2$jiuN&%m-=}*To6^=RsQWyDC|PsH>0jxbh?_Y~X4;ZGb}+HF;aiGh8a#@j|N zaNmX(my}90!k26#e^WoGcJR-3KdSuMq01+KHlJ_kvb$_~`k2bo7peUMuM> zkfRUl2=)yMF2n1;QtDk{5lVyp$iCk(r#RhVN32;ML9wr@$o3T|D5ZbkA~`V6lD%Xa zbc;Y^fgEbu`j_S;cT>ZE_(tJbKYJvTU9ObmBHG|RZ3ZL<8w&(2(kX|QuYM}ml9eSx zGjGS9MJdT8v`;z~73jFUad5-96aOr?mqyA9Mf3nkzT%o?F{c?9QW$QtatVdZngh#H znoDVE)94`snqvnF1zmhW4jsQzZBldNd97y0{Db$7ur;eR+vtO2U`3&h&;H>)C%i?% z@iA#`?xtsMUdB0gqu?1VRXjVYEJ@q55bZx&A_8lzawzzUyTIHQoi17@?ZPWkq7l&14kMz2Ypo)DiA29X ztInioLvg1%svx2RD+>%b9l2x`(cPtTX!b`tO^P;@cPi1yXwPXQqGP*DMSRg1IrL%2 z5tE|R_d}gQohGCmYjwjzvm+)JjS-F*iEe56*raGffu|C!q;#{&=_1+n;^g>WX&zY{D}@qUYAUW>R!|dgu)A zREHLfNPT~$D|*xLgSdCVCVcPRCWp?>+nRIHdXYz~!4Kwi(p$~9!;Xj=)&BQx6M^cn za;Wvwuu0X1GEa4MQAW~dEZR4rr#e_YR)o7r^}3VYCRG~>J(X%@FulIOi0Vl9SP^ab ziX2+7&ry`?eNiX%G*KOm8&Vx>_=D>81Nv$G+8x645s7wPIgxYGI>G0tO^HTY$NxcBv?lkwAKf8*Zwoo}@tI2|MH|XJ z)kiMmUU+qC`^aduMz#NKAp-Nq$)U&x3yaMqh@sq5sZPoqJjNuy)lKX9<3#ufsou2X z4U?+V_eGuDQ>Aq&#?_6f+{;FF|*sXE>3>~arj?FgpQIvcMOpOy*7t~=#WyYnNHstx6y>KLP>^%~XQSStHgXwExn z$JV><6fXB&a%f9$d5O8bF7WOXYZR}Y`l)NepN)5 zj+a9#<_(xso$j?x?%~Yd;3i>%>he_XWurQ_bi9ank?O0TEV1S~L3BRos*W+r*t$($ zwI)ImUE_sgvn+=?qi^P1wNCEgI4)f$tT)TTHR~QZwBx{ zJse_EqUi+nx09lin~QE(&$>r=_LFE&<8hOs4dtFnv@*2*p{{66?0NU!BYZ3Gl|wUE zeQHv4`mU%Gdz!EgH>YARqtzPK{+0KNzy(qrTli9`xn0pv?5R}al!9+oD)zEb9lUU_ z2ydGphu-;ol}XjuiC z?%@!V$~{tDHYEDmh4A%q&sqC*;rfI`zgstZjJciBQ0@&AO=qqDASpU&{4mj)1aW`z zb>RtqLk_K}T47Rj`o5@>d#VpH%E-Drm3!H!_6ENpe6ERdXrA*eld28no=UYcwBBn- zwcj;S1UgA|&xO4vRi}HclY6Sudr0f}!c^{M<8`oeq6n{_B!~7MIb~9{q1;m)W0axw z0Yj=I>nDk5KdD~7`}3Tu*2z7b-%Aax$I8BCsocxx^;^S#9TV%HB;ui;kwY(STx2sB zA%=2Kb&SC&1jSnuZ*!J=VA>`xk-e6MmS!%PY(62SZ7kUq2z|MG3BxsOj*9LN?-QO)KP!iPpLCei zY$)|qAH0--wOey)U01Ys)6WXu%%77(FGmw5MW-8`UFa$2tPdFx?VtH`BJe(mPVB8P zIe*ho=BbV^%D7s4E>G9#;QK!(!mm!2LmzDM=3KN+A*fv^HJ@p9MMqwpETZQ~ z^johtnG|g(@l;0_WmLUdGxKe@Xw9H0c5bqWZ~b{Wv}9q#q-aBVrxLB?^8VDM3QSjs z-<=%04kn8xJ`!7hUO47Xkwbm`XHBY3-xGBr4=FupCuGtLz2W8#|4t*?Id_V1wUg*W zT{Fj;J0cp&Je6o?k z*O*joDEOdy&`zYaY`y3kI=7en zJ^A3L#jIrePyLDrZ2XoSa?YK5ySe?*Q1+>gGs@t4bLxabR2G1-wXz9pjj?w3QmPHZtLJAIGTNk2_y$D3296tZ$Vw(ousU;e9dX!YLr zP0BVDe=6C^5PO?ym!#jN(+NM_V@+cDuL?&G$u8e=#-!|Yw{`Llr}$K<9hjUM~tHRazYjWs}_Y0G%4dtKeSfdQG|5;zPHnkgnO?b*5kVB!$SMD&EBk8Jj;!kyo z52@|gklI8to2`-UEq_4x4v_4%{q-hg8;U>G!3Oacy=Y9f|G)zx;Qe(uRMFL9QnsP^ zQ^`(FP8dAcWaW0y`|BbcC)wxrcbb%)zC-H7pDMMJzwk05I}-nO5pDhrIrQG!eI{ia zioap9X{5b3mD-uz)(rn*&A%by3%)If-dtNL%dGaxCRL{giOz3!aja2=w&Q`M>g0y5AKFL1EnI6KltUjmnscsNC;n6u z`IMpdzv-*iq@R23gTix(REN(WG^yH9`l&vQDMRg{<9cSZHL|^z9u&UazbS_{9=>2w zwxRS>$wp{<9I3%yRUoq zS53+`lzuANIM0A;@nC3Y<#y!cZ;9xJsdDJR^35h?8%jUbX}{zz8FgiA;xD#gs)%1D z*_&V7XHvGI_`|U#6@L`ksWB@)=J*`ib37W=iK|nEfez=mpZSQRBb5!R3B@Ukv3f!VQ>@4O18WDcZ8>#WFKx_ zTxKpu48@;HwldTX8I$ep{vF|K_+2@)wPC$U+37o^PW)+7JGvkhe_6ThZ}?pinEiWl zsM@>Bq-;a+r;?3bFhIXvGP02bXaAlEM@aVPFC8-}JKb%a!JlecA5z;>l}hcb+>S(k zPeeTr$)RJPT{J1%Q2eP5HaOGZ*`ShrZJOWkB+>k0EarJg#Cu8h+aJ!q)7&nZE?Y1E z=;Yt9X%>TxxBrtIs@ea#NzsPlZ`h%RPHO*Ya>LH7S{vGL|0m&^ z|NC<2i|yM?sy38X5Sha&p2T@IZ-{8X8pRHkRO zMz#CO!@|@0M{=mU+Z#|N8)jGkan&hVp zvT3xc=e1@86?yABBI^8OIdp#W$0k)9%01OlM#*P7rmJGmGcg(7$z4^mJI0)UEaG93 zz3;VoUp2Qo8cIIMR_RQe*r&fcCc=L#94r4s4%NT(nn~5^yQ5C@4a@5yIsCWD_b`~= z8r7Oj#JTcMgzEySUiev~N!5m;PxUbjrx<)IwZGxjm+ijrC&IJsKg*%`>99%J>1OL> zpX!`n=HJZd%l2;j&%!tFPvy}5aJNa>hQd#Epiu_d{?rbc)kfl<_opIoh-CY>JI0&a zA=72+q@QZCpEAg%+HPu`r~F1q-|gU`KNaECencxB1ydQ`YNN%kv; zE}4{_?zT<CZ3zlny zYQELz_OAbP;p->eN9y0mxo*7(9Gdi({AHL%sODRZZh!xui$LhV(8aH-cbL>|C<9eT z9A&sYD|MnF86Wz(gQ5Q-!c~7kSHLbmWKy@G5LD?_2HYWox+7J8A)=>A_mz)(P3lhH zFLhE-8{R{|X;OagK6ElLX}>nTPydC8`~NFl3j28ZJ?8dHLoo>5>M{4XlDlPcc%|7) zG_n)^|0*0a|B|kTy{E>c>~yh`|+SHgFWbe~+% zWm31H6jU8`kb-f#>hj?BoSonPbAKfQTOXrKV_&{%Qn#TLROwFsGS86i;MT`P_{G1b zt7D&Da<92GN#8AXQc#uN$@AIzx;1eUdGW7B^dRY;9e&fKZbK;u-Ki9$UrPSg&}Kqg zJFUj2pMtT2e=Xv#Pp3;{Z`*BBcDmi!rJ!|`ENzE<-gN4v+K{B)Sd3PP7LCtV5YP3nnPz&wdR-;Yx(aYzWDFxn%T={f6d%p zX($DU9doG0{+(oak04tc-i!ZEI8Kr5wmB5J?9DG zI6$)7Pj#4-Z72m*AJCMc_p($9W~bYE;0fXK{v%yR`~8GT-G)+7rCS+#A2+Dm?fpmL ziIeW7+bX`1>%CGZ1vTj%_NMmA?ELn||55mw|0i8cyVh$`x1khN=~jl`XAJ80H~&u& zSnyrCoVL5kq;5kgsG1gt*mJE^gx8N#vqf6)cCcP=w2+E4TdzS4nsoGHb4O4yX1fT)c-jh!X--dspYiiFq zX;O8%*E%CWRrV<-yDw&*VVC@O&}K)(kSEY#>_M9lBzWy z-h)^FNrd5t6Q2C-+pR0+oQKnWL&099rcgDO+Q9WbgkL(PjTk7u61} zF)7R3+@x$n*{70?^9_Lu#$>x^ z|BLWMNcNgbFaC_V9Wvc*o$yml3{0M6HR856@-M>Yd77@QeQKpi+37i=^9y0skwzJ3 zr`3NozdqA!B!17+BG5~+d)l_-T((a9sZIha!|dH7%&OON(VkiMs zpT+QL%R{mT{mo9d^NsHb*QccW{j>EZb*I~{6M?ETfJkl!tw3%_xBJuY3C}zKhpw}I zu*IZqLm8+#;vfT~v}Up)-QIWpkMOxJOZ)W8u=4QqBw&Gzko`8g3d zLAn#``%LOKl!B^bjxyrjpE^e}!uG96!QhGKMEK2rr3-E^t^8SY`=y~2gzi)d(yu2| zZD!3G!I8rI&3_fq&q?-MvujPtPTwtcQcx4#>bWJ?)@#*lB^tM5pZ}|f@A~)rVkg7FfhrBUJOP zMz?e6^TO3dx(~g3!K7|OA*ed&D8p{LYOY48=39+!ch~d6^VYx7wYN9T`8jhtWx8&i z5LC?!M0k7l8r1E5>)(XW`F*NXUDs)G(f(BVpjV0PQL-}!wJ2$Syqip?f< z8wx>{Zp|6(5p-+LJOsnv7vY*8&=t7*-ZQB?eZSO6L47j zCz^gJ9P|F2F2cQa@#I{GiO#2C)#o#1|Pzb6HI?BlVkU`zv)jtxxi=_Ltt4B@hHWY#? z-3V{hx=q>n?Z5aV5!m_v=u+HgFPhYyzF+Eupenp^{$bdK_J&<9Fq!3=7!2nDHdw z2(q>5ef-BFzV1KhTHNcmnUtMAq|!-2)#o#1D(!0ca*wA zF9=VS!$Mc&{>VAS+-_+o235k9vG>qbc}5VvWl;Ft5H3o+BwF}RL;2dH)h3mv@0mJ5 zs0nb#+Uq5XM!BCvi-3P7DX)0fZ&JCTC{!s|B7Ep%m2xg+XtrQ73zHkmpyxpnEy4~b zDL?0LHK{z^bDcEA=|NS9hwn&^tw#79p%ES-%_75$iwjJ%omDF ze)`ntPLO2bI0oHS2f`+G8%jf!?&Nqx?-nI z-JA>8$w8bkRK+(<7|eL0B5AlrcZ4L1sBZ!3KGt!-@SKJ4=)4^!g{M2NlZC38gvrw=MudAw zv+!+MNWyo1+HO*~p)gb(e3W5$+X$2LlA)s6#{HyO1m?I%xc6jl&V}oQAx<0)TCAk_ zbusLAngbLs=z2$pM< zCrGq#G`mT8t*gzXazlBjQjQOkzFDb{_t`0Tl4#*tu!xkuc;!Qr%F{j92}ISrLL|8> zWKg-AL<>&`ly{%GYErqOL{uGvkcjaGsS^{~dG00A!nbBIDc|$Kl3y{GEa}SiB5~-{ zq2?UQ2+Fl#PNGGi56bIyylGOop-jY4D3yuyJI=D9Or*KALnjhP$iyIN7U5k>NcjA9 zyG;r=l!?QH7s%m1ND5DWa@Nnp2x%5kcNGagUv=E1@bvvtClgg4f|N0Nc`6gLQywGH zA|8eEea=r!DmRpgD&@+koR%5Z%mdQ*JVBy`W5X+?yuNqAx6JLI>7MIkqUyw9^3 zZ&@nMvs3OR(ZaWE87bet@qkI?hB8rg9KrxsHD3iAk7`NcWGHGr@B2x#2%Ld(|EuRs zDmN60P_B38O|yS$jzU3_EW(?sN%x|8v+vLK{;89RsxTjZh4rMv8sVB@VuUn{=&a== z{F6_Xn-p#+5>+4Kl##hR6^Yp?kCA8*-w)*nqnk`B|FV&&oFhMEP@VE7yrc)x<+(XnxY5ayN+UquZmKeBMqubPV%Ly@Rbu7vr}a*_Ba$Z(NDO(^+#vk2F&B;m`?1WgJz6pAY0 z2u06dTt~Pz%}KL}egffFA{{1$r#r4QOjMmq)O>Cm!E%lA7>O3~pof$nZ%&w0ZYUE~ z$022Cema%r*(pzuXyJJIHB#POSMh7+_Rn`NhNuk9*Q7ErJLOIiEnFv{d}Wo_ zq;f-B*w=(OLB| z4w)G6rZO=*^U77k}EDQ}s($E0#YnW#Dr zDP!}^sZ7jHxsyZ-R~X9Q_#|plxuHx{DObkk{RWl0Nwo0Pyg|z6MlYLGp1y(|Xii=j+ZDObkk+fsX|=38Z`P8T)QOpNoBXc1^yL&{sX ztTCxP-E*B##0f=Jng=FlJnfP^nrdVRNwEmeT}!gpy&f?)1Gt@0-+}?sv8+NGHm_ zr_MxQYI&0pbo)uN2&{XPbVrt)F{#^74C1(>N^i#l$?t}rd>%n}kR*%nXVATA<`+5F zt&@YQX^6^D{NMC-Yd-r#NV15&TTi+dU#R-FxjZqHgQ`z$%1E4!^Mm$hHe4e-Mw&%@ z$p#Ys>EU{l!VLwXN;m@C;}~I9Tyjg%2v3k^;W!Q9?Ymk`3QylNb&_ycf?w+@8*$u8 znuW{nBjH5l7Uq3YaWa@cJ|xR*2w zUmJvf_G0CO=5|g)VTgmzpi*VjJvn(|0uke=N)5?J*v9>&Sp-)2NqEP_)j1cglZM0Q zA=3HrhmyiaICrZ#F%cxqBHRPv+fKY~Qn;ZwRDJAGM&UyjGR#hSghY#I<0ewRV(&qd z$_?eAO1Ux+52dz_?3Bkyw1}6#Mar)>TrjCTefQJ}L`{-M7o7MIEqG~20vfPutsL9}Ou#(ZD z*+bnVT6jK&a!>zTCY2k?MAcCUndsS&I;EmHlbBSl9g>r1;oH$b%4c@HmviNMp-86= zGkxaKoJUOR)(lwuBv}Mr+CsWx`%jzHZ736m9fb;1R|_0LxMoucl4cP;3gPd(eZ{13 zLy@TZ?52##7p5XHJLM4)Euw21N%^X^i+{^pu%z#vI+3Ue^S}Xv%3~y2#4kbloF(f` zDmN5~D&@+sd|4{Yv-3PbqJ^VnD=F`C>@ul5-E*BtR82%gm^(TRDtD4-;ac28%HKVI z)TDAlk*GQhDFbtFDiX8v+)bi|CkEwf4qwc*mzWnBaaY@UJ@;Q zzHOv@-mdvm&F!IvLJ^0d)X=;@_Kiu7aFb`{G{;YEnv-S`7=ZAe4X>LNZYUI0!iRst zA3?Y_%}KKezqg%)?_Ivlr113pQzsKuX|5SvWv4tsqD8d&ZBoAe#Scs>Hb2kTNvyF{s>0qJ?YD4pRPpYjw_*>tv#8CZaMl9~$6h=ee6i3(p5o?rZRyRBk8} zRmUME&4-Tt_$P=CT@!JnG$+x*_ga9IzwB)_soYQ~Lb)o<<98=_(4Yc>e8mglm!|NSa0X?VTk2gU>!QDcn#dsy@cybDM8vDicSr zT%$ZfqD6Fmkd(jm;mf~cE?Lqo*U3cH*~AfsSDKAQqdZ2UMf?bqFWJA=q;f-7A}!X}j)%0$(12$>j;8&n=6(IVdZ zE-8O?LwC-V>t*85OhnDVFgwo^Bw9F@g-H28wd429?VpA+QFRaqE^GM+?FZaEs+eo`y~`yt!edcvgabi;LqiK=Oc z$nx;N>dMyqk|IcoMR?_2l6`9PC6lrZ<)P{`oH8zN%sihjX}3mqgd~gT1?b-O+QNs- zWlFkkoitRPNlczP%=EjK5p>5$vWRbck94no=?#;*4P~L~_=7Br@63FiyAgCJNV0HL z>?7UfpEaA*Z72y8GmEixswKI>4;ymMcz-v8{ z4929!w2iSxY}2&nPVJG)Xxm(E+B7|rHi=2wq}tl1NqS{jPC;M+0Re?&<&^tUQPE}P z^5B$9E|*L08wCWR&-;$M`^>Z54f7fQz|H>h%=3KT=Y8jy*_n6VG3QhUcMu>8y7+S7 zF0C9=$Stc1GHy#1gxj7^TEF49shHbD_Dek=3mFrPJL!byQ_A*B+k7vq46=m;VdmkD zR!A>Co4M(!I3NoKV+C;gBt|LZwyh@)o=`!@OMWV@wlgoz4!9ARg-UHDfcFOg;?Ni_@tW7XW(YF??g)Wn75yd9v05&tG2&rj%3NG_`;B65CS4i72cxbd?v z@-cuG)GdrWDtJaAxoyrJ)I>BDkyUeTisdr%!pJoMEohUQfqZ^0=((F+S{(drM#mvu z&8;sDVy6NI>zW{TieeBz3%Y4WZt6R9)5#qaMRoxZRdefVt9EX#ki(E3n1zgz7631( zGb#j^RTR--h!sV{>*i5z2b|QzEMOK4A*}$eH#RB+msJzduQ+m&vM_QZKns-}jNC14 zOd+}L{^_75qH0cd$c2$t1GHeuyaeQ(;XYfG?Vqx0qJ$h~*rr=*;=;(=0a_TDXXHg5 zdWGb+IbT>!55zFBeO zOu!Df9+-uUqwN5GCdc)8Wjm;>qKLp*HCMm$8Q`><1G8Z8>;UjXx^RWyw)>}pnkcE} zOV^s<1wv$+4M{m6Hv+U!S;NSE_hc$0msJxHIX_7mlu2F<(1JOB6BkB42GD}~R2Pt^RZQJ70r}z7BMQl7)kJh0;^*d7mTGP%=loV8TuUF)}Oe{a|J?W>o)w&VRB zJ3rd7Hu2*XF3jyJRpM5GE#GcBQnh5oI=0E3E6i3_bNHpdikUA{F;?{pj=rfD386=F zo!8w4{{g317v5U$=e*JT{&lyjir*EWqF*)(f+`xOxWIm^dHaTmpbE`#CW>F=BBr=n zZQL}A1sHiqxz@v#d8_B)iX{SM3fXiYUdpyYHq8f?5v9sq#kTUW#wzRQ3i4A_y_|T? z&s)J13>xdGNd=K&L8T5Ikp#02EaRs$>{L&Z5R(-`^WD4r{?$mop ztV?^>6QzEJE68(M`+dw0YP;dg2Bx5v9?SR=iJlquWfDDYjw`6`30+@d(NVkhpZ_XT zkmr!>`$=@~h>aw=#`S9?`gAu}ke~7LuQMCe@58wd2m_Mnm-l~z?UktQjcyOJy$QAb zT;GF?5ZM$|eUmUBJNhl6WJhme3Tk_@=R-s}IQS50MMTZR%m(Er;P@ji6kkuAt~+b&oR}r1a-MLHxAGKFJi+_FCU>GX-U}VvH+@r}*M#mQ@tJ(C{6m zpjampzDqKn>;FBbAkP_BxPqd`Hh-UZ)}MZgFoz#{nh*vDKEt*`{jQzj3d*Y2rDvH9 zYQ=ofb4)?e$5XZtoBrVESuE7jo@-n|t!QuC%4|^En@YDaKS-%f{{it_7WzY`Ai~1y zTtS2X(V1=x(l{wg;EL!fa5x{BwUyw({Kn z6Q-c(UT&|F5kBAhD&s-zx>EU5;(6rA&zORucSgNNY%Y2HoGB<9^#fc%gw+>bXEr|8 zY}6k61=*X<#{81(O$P5@F+vnQdzdRI)}gvzGaD2uHvcz7iHiL#Q&2C5`M$w;&^X*X z_6D;-o_#OA$rO|ySHtf}OQ++1Pn1!=x0r&);qU}kQ1srWx0wx6I*jiSKTS#7nS$D0 zp?#MrsHJBo-xUTW%%@x5V+x9HEcydeP^|ovKQaZiD>wL0OhKMAuWQ}&Vgh9AzOvU{Fg-E>-{NHP}}{7KV@54+U271tW!HlT)U_=`L0~D z#CK6m9@J2)aUVA>S+eG$8WRdH_FDCPfLk4{L>2dqV~hs(+Cm zmynfb3cS@BtdKMp1n*16L%~HgJ1t6<-f&S}HNiW6p*v7=CQ1USuYojpxhwcWnJ|R} z)9z@V6(s~w+X1i31oxKl=%T7Q8^R?7QIQGnNtBL>l6~+RKPi3DyeJ`vyvzHz70P{1 z8NAaCd?AR`oCq!>8Xy9E_Qi$kYQ9{1jXK&7)j ziB7^0I ^F$&VK(^|dxVCte8Y)Il#id70R9O4_sTcyJZ$y`EKE&i}FGWbfgTDe{o zr8ujkZ%O4=Nmi+(5LU{SLUbj=Qm!B&E1zIkBo!nCk#Y?r=rm+i-UbVMf|P7^KUNA0 zPr{f$5P=!lTtdDw!(hQdutJ$An-e8uHP{797G{p%l&q6j6x% zlcI#I3R;UPL`P8xmynfrYAKh{m`LAyhD!*dat0*mA=Dis?eMH2BnOQNzq9Z-oOBeX zXTpDJvn^bpM2feL{7F*gt< zl}PJE{S-*h+h|P8UV>>5AR&mz5)&O0$J5%lgfdYd0(X|BBWH3}lu)w|barqn)F8vT zPA;J_(Vo@CQ-~yw2)IWh&1$@BHwFC5&+i zbw`H(IJK(1B1#BivU!49?JvB-r8MhcDo(yiAub0^atZnJx(1STV8*pgaS2(Klupx# z1L@bO6c{$cC6tNWIZ;AmwYh7SLR>$8ouVGdnWGS=BIl{prcpaHdNfvd_lXjMNUd_^ zR!FKks^Jo9<9M{2GaF2(JG5TzTtZ_aX9y&Sh&HUN^`H=QCp>9HeXJLk(6~SC>&+$P zzGqC75QJyrZf8;f5(|8|gdj>1eVy4TM85h0_HYU1K44Om5JXz5AGboj%8UHDgdqA- z0;rYGUM-hU#MBv(AcaV(Y7gWR%KgNdAS#7r26G8P9No8pi*R7B9~AmPK6}Vd5CX_P-D*}a|u}$XQfc7Cp?u) zXiR8b)3}5na(YAwC8wz(opT}gvxhUdgt{XkB7yNN5qdu z#gE6tkH^K2C&Z5@H#n;n!QV3UXa3GU`QIj8)Q6iFtJuG5*uSf9Q#s$bK^kA0mht{( zxO5Qp;f9swTcmL7)GCR0NttvXx4Os9jZ#BYq08nwxcggGJpom + flink-orc flink-jdbc flink-hadoop-compatibility flink-hbase From 8ed02baa1d5cf5631d8c73943e42ad331d38b7b6 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 13 Nov 2017 14:54:54 +0100 Subject: [PATCH 2/2] [FLINK-2170] [connectors] Add OrcRowInputFormat and OrcTableSource. --- docs/dev/table/sourceSinks.md | 49 + flink-connectors/flink-orc/pom.xml | 89 +- .../apache/flink/orc/OrcRowInputFormat.java | 745 ++ .../org/apache/flink/orc/OrcTableSource.java | 455 +- .../java/org/apache/flink/orc/OrcUtils.java | 2379 ++-- .../apache/flink/orc/RowOrcInputFormat.java | 241 - .../flink/orc/OrcRowInputFormatTest.java | 795 ++ .../flink/orc/OrcTableSourceITCase.java | 134 +- .../apache/flink/orc/OrcTableSourceTest.java | 266 +- .../org/apache/flink/orc/OrcUtilsTest.java | 148 + .../flink/orc/RowOrcInputFormatTest.java | 472 - .../test/resources/TestOrcFile.emptyFile.orc | Bin 523 -> 0 bytes .../test/resources/TestOrcFile.listlong.orc | Bin 627 -> 0 bytes .../test/resources/TestOrcFile.liststring.orc | Bin 1298 -> 0 bytes .../resources/TestOrcFile.testDate1900.dat | 10000 ---------------- .../flink-orc/src/test/resources/decimal.dat | 6000 ---------- .../src/test/resources/demo-11-none.orc | Bin 5147970 -> 0 bytes .../{decimal.orc => test-data-decimal.orc} | Bin .../src/test/resources/test-data-flat.orc | Bin 0 -> 408522 bytes ...OrcFile.test1.orc => test-data-nested.orc} | Bin ...tructlong.orc => test-data-nestedlist.orc} | Bin ...stDate1900.orc => test-data-timetypes.orc} | Bin .../flink/api/java/typeutils/RowTypeInfo.java | 17 + .../logical/FlinkLogicalTableSourceScan.scala | 16 +- .../table/plan/util/RexProgramExtractor.scala | 12 + 25 files changed, 3306 insertions(+), 18512 deletions(-) create mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcRowInputFormat.java delete mode 100644 flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java create mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcUtilsTest.java delete mode 100644 flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java delete mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc delete mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.listlong.orc delete mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc delete mode 100644 flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat delete mode 100644 flink-connectors/flink-orc/src/test/resources/decimal.dat delete mode 100644 flink-connectors/flink-orc/src/test/resources/demo-11-none.orc rename flink-connectors/flink-orc/src/test/resources/{decimal.orc => test-data-decimal.orc} (100%) create mode 100644 flink-connectors/flink-orc/src/test/resources/test-data-flat.orc rename flink-connectors/flink-orc/src/test/resources/{TestOrcFile.test1.orc => test-data-nested.orc} (100%) rename flink-connectors/flink-orc/src/test/resources/{TestOrcFile.listliststructlong.orc => test-data-nestedlist.orc} (100%) rename flink-connectors/flink-orc/src/test/resources/{TestOrcFile.testDate1900.orc => test-data-timetypes.orc} (100%) diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index 0b4bdbede0d3d..7387358beb931 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.md @@ -47,6 +47,7 @@ A custom `TableSource` can be defined by implementing the `BatchTableSource` or | `Kafka08AvroTableSource` | `flink-connector-kafka-0.8` | N | Y | A `TableSource` for Avro-encoded Kafka 0.8 topics. | `Kafka08JsonTableSource` | `flink-connector-kafka-0.8` | N | Y | A `TableSource` for flat Json-encoded Kafka 0.8 topics. | `CsvTableSource` | `flink-table` | Y | Y | A simple `TableSource` for CSV files. +| `OrcTableSource` | `flink-orc` | Y | N | A `TableSource` for ORC files. All sources that come with the `flink-table` dependency are directly available for Table API or SQL programs. For all other table sources, you have to add the respective dependency in addition to the `flink-table` dependency. @@ -485,6 +486,54 @@ val csvTableSource = CsvTableSource {% top %} +### OrcTableSource + +The `OrcTableSource` reads [ORC files](https://orc.apache.org). ORC is a file format for structured data and stores the data in a compressed, columnar representation. ORC is very storage efficient and supports projection and filter push-down. + +An `OrcTableSource` is created as shown below: + +

+
+{% highlight java %} + +// create Hadoop Configuration +Configuration config = new Configuration(); + +OrcTableSource orcTableSource = OrcTableSource.builder() + // path to ORC file(s) + .path("file:///path/to/data") + // schema of ORC files + .forOrcSchema("struct>>") + // Hadoop configuration + .withConfiguration(config) + // build OrcTableSource + .build(); +{% endhighlight %} +
+ +
+{% highlight scala %} + +// create Hadoop Configuration +val config = new Configuration() + +val orcTableSource = OrcTableSource.builder() + // path to ORC file(s) + .path("file:///path/to/data") + // schema of ORC files + .forOrcSchema("struct>>") + // Hadoop configuration + .withConfiguration(config) + // build OrcTableSource + .build() +{% endhighlight %} +
+
+ +**Note:** The `OrcTableSource` does not support ORC's `Union` type yet. + +{% top %} + Provided TableSinks ------------------- diff --git a/flink-connectors/flink-orc/pom.xml b/flink-connectors/flink-orc/pom.xml index 1ac7eaa7a9f2f..3ee5e49331878 100644 --- a/flink-connectors/flink-orc/pom.xml +++ b/flink-connectors/flink-orc/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.4-SNAPSHOT + 1.5-SNAPSHOT .. @@ -40,22 +40,39 @@ under the License. org.apache.flink - flink-table_${scala.binary.version} + flink-streaming-scala_${scala.binary.version} ${project.version} - compile + provided org.apache.flink - flink-streaming-scala_${scala.binary.version} + flink-table_${scala.binary.version} ${project.version} - compile + provided + + true org.apache.orc orc-core - 1.4.0 + 1.4.1 + + + + org.apache.hadoop + hadoop-common + + + + + + + org.apache.flink + flink-shaded-hadoop2 + ${project.version} + provided @@ -88,65 +105,7 @@ under the License. test test-jar - - - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - org.apache.maven.plugins - maven-assembly-plugin - [2.4,) - - single - - - - - - - - - org.apache.maven.plugins - maven-clean-plugin - [1,) - - clean - - - - - - - - - org.apache.avro - avro-maven-plugin - [1.7.7,) - - schema - - - - - - - - - - - - - + diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcRowInputFormat.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcRowInputFormat.java new file mode 100644 index 0000000000000..4353cbc66dfb2 --- /dev/null +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcRowInputFormat.java @@ -0,0 +1,745 @@ +/* + * 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.flink.orc; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.io.FileInputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.orc.OrcConf; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.RecordReader; +import org.apache.orc.StripeInformation; +import org.apache.orc.TypeDescription; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.orc.OrcUtils.fillRows; + +/** + * InputFormat to read ORC files. + */ +public class OrcRowInputFormat extends FileInputFormat implements ResultTypeQueryable { + + private static final Logger LOG = LoggerFactory.getLogger(OrcRowInputFormat.class); + // the number of rows read in a batch + private static final int DEFAULT_BATCH_SIZE = 1000; + + // the number of fields rows to read in a batch + private int batchSize; + // the configuration to read with + private Configuration conf; + // the schema of the ORC files to read + private TypeDescription schema; + + // the fields of the ORC schema that the returned Rows are composed of. + private int[] selectedFields; + // the type information of the Rows returned by this InputFormat. + private transient RowTypeInfo rowType; + + // the ORC reader + private transient RecordReader orcRowsReader; + // the vectorized row data to be read in a batch + private transient VectorizedRowBatch rowBatch; + // the vector of rows that is read in a batch + private transient Row[] rows; + + // the number of rows in the current batch + private transient int rowsInBatch; + // the index of the next row to return + private transient int nextRow; + + private ArrayList conjunctPredicates = new ArrayList<>(); + + /** + * Creates an OrcRowInputFormat. + * + * @param path The path to read ORC files from. + * @param schemaString The schema of the ORC files as String. + * @param orcConfig The configuration to read the ORC files with. + */ + public OrcRowInputFormat(String path, String schemaString, Configuration orcConfig) { + this(path, TypeDescription.fromString(schemaString), orcConfig, DEFAULT_BATCH_SIZE); + } + + /** + * Creates an OrcRowInputFormat. + * + * @param path The path to read ORC files from. + * @param schemaString The schema of the ORC files as String. + * @param orcConfig The configuration to read the ORC files with. + * @param batchSize The number of Row objects to read in a batch. + */ + public OrcRowInputFormat(String path, String schemaString, Configuration orcConfig, int batchSize) { + this(path, TypeDescription.fromString(schemaString), orcConfig, batchSize); + } + + /** + * Creates an OrcRowInputFormat. + * + * @param path The path to read ORC files from. + * @param orcSchema The schema of the ORC files as ORC TypeDescription. + * @param orcConfig The configuration to read the ORC files with. + * @param batchSize The number of Row objects to read in a batch. + */ + public OrcRowInputFormat(String path, TypeDescription orcSchema, Configuration orcConfig, int batchSize) { + super(new Path(path)); + + // configure OrcRowInputFormat + this.schema = orcSchema; + this.rowType = (RowTypeInfo) OrcUtils.schemaToTypeInfo(schema); + this.conf = orcConfig; + this.batchSize = batchSize; + + // set default selection mask, i.e., all fields. + this.selectedFields = new int[this.schema.getChildren().size()]; + for (int i = 0; i < selectedFields.length; i++) { + this.selectedFields[i] = i; + } + } + + /** + * Adds a filter predicate to reduce the number of rows to be returned by the input format. + * Multiple conjunctive predicates can be added by calling this method multiple times. + * + *

Note: Predicates can significantly reduce the amount of data that is read. + * However, the OrcRowInputFormat does not guarantee that all returned rows qualify the + * predicates. Moreover, predicates are only applied if the referenced field is among the + * selected fields. + * + * @param predicate The filter predicate. + */ + public void addPredicate(Predicate predicate) { + // validate + validatePredicate(predicate); + // add predicate + this.conjunctPredicates.add(predicate); + } + + private void validatePredicate(Predicate pred) { + if (pred instanceof ColumnPredicate) { + // check column name + String colName = ((ColumnPredicate) pred).columnName; + if (!this.schema.getFieldNames().contains(colName)) { + throw new IllegalArgumentException("Predicate cannot be applied. " + + "Column '" + colName + "' does not exist in ORC schema."); + } + } else if (pred instanceof Not) { + validatePredicate(((Not) pred).child()); + } else if (pred instanceof Or) { + for (Predicate p : ((Or) pred).children()) { + validatePredicate(p); + } + } + } + + /** + * Selects the fields from the ORC schema that are returned by InputFormat. + * + * @param selectedFields The indices of the fields of the ORC schema that are returned by the InputFormat. + */ + public void selectFields(int... selectedFields) { + // set field mapping + this.selectedFields = selectedFields; + // adapt result type + this.rowType = RowTypeInfo.projectFields(this.rowType, selectedFields); + } + + /** + * Computes the ORC projection mask of the fields to include from the selected fields.rowOrcInputFormat.nextRecord(null). + * + * @return The ORC projection mask. + */ + private boolean[] computeProjectionMask() { + // mask with all fields of the schema + boolean[] projectionMask = new boolean[schema.getMaximumId() + 1]; + // for each selected field + for (int inIdx : selectedFields) { + // set all nested fields of a selected field to true + TypeDescription fieldSchema = schema.getChildren().get(inIdx); + for (int i = fieldSchema.getId(); i <= fieldSchema.getMaximumId(); i++) { + projectionMask[i] = true; + } + } + return projectionMask; + } + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + // create and initialize the row batch + this.rows = new Row[batchSize]; + for (int i = 0; i < batchSize; i++) { + rows[i] = new Row(selectedFields.length); + } + } + + @Override + public void open(FileInputSplit fileSplit) throws IOException { + + LOG.debug("Opening ORC file {}", fileSplit.getPath()); + + // open ORC file and create reader + org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(fileSplit.getPath().getPath()); + Reader orcReader = OrcFile.createReader(hPath, OrcFile.readerOptions(conf)); + + // get offset and length for the stripes that start in the split + Tuple2 offsetAndLength = getOffsetAndLengthForSplit(fileSplit, getStripes(orcReader)); + + // create ORC row reader configuration + Reader.Options options = getOptions(orcReader) + .schema(schema) + .range(offsetAndLength.f0, offsetAndLength.f1) + .useZeroCopy(OrcConf.USE_ZEROCOPY.getBoolean(conf)) + .skipCorruptRecords(OrcConf.SKIP_CORRUPT_DATA.getBoolean(conf)) + .tolerateMissingSchema(OrcConf.TOLERATE_MISSING_SCHEMA.getBoolean(conf)); + + // configure filters + if (!conjunctPredicates.isEmpty()) { + SearchArgument.Builder b = SearchArgumentFactory.newBuilder(); + b = b.startAnd(); + for (Predicate predicate : conjunctPredicates) { + predicate.add(b); + } + b = b.end(); + options.searchArgument(b.build(), new String[]{}); + } + + // configure selected fields + options.include(computeProjectionMask()); + + // create ORC row reader + this.orcRowsReader = orcReader.rows(options); + + // assign ids + this.schema.getId(); + // create row batch + this.rowBatch = schema.createRowBatch(batchSize); + rowsInBatch = 0; + nextRow = 0; + } + + @VisibleForTesting + Reader.Options getOptions(Reader orcReader) { + return orcReader.options(); + } + + @VisibleForTesting + List getStripes(Reader orcReader) { + return orcReader.getStripes(); + } + + private Tuple2 getOffsetAndLengthForSplit(FileInputSplit split, List stripes) { + long splitStart = split.getStart(); + long splitEnd = splitStart + split.getLength(); + + long readStart = Long.MAX_VALUE; + long readEnd = Long.MIN_VALUE; + + for (StripeInformation s : stripes) { + if (splitStart <= s.getOffset() && s.getOffset() < splitEnd) { + // stripe starts in split, so it is included + readStart = Math.min(readStart, s.getOffset()); + readEnd = Math.max(readEnd, s.getOffset() + s.getLength()); + } + } + + if (readStart < Long.MAX_VALUE) { + // at least one split is included + return Tuple2.of(readStart, readEnd - readStart); + } else { + return Tuple2.of(0L, 0L); + } + } + + @Override + public void close() throws IOException { + if (orcRowsReader != null) { + this.orcRowsReader.close(); + } + this.orcRowsReader = null; + } + + @Override + public void closeInputFormat() throws IOException { + this.rows = null; + this.rows = null; + this.schema = null; + this.rowBatch = null; + } + + @Override + public boolean reachedEnd() throws IOException { + return !ensureBatch(); + } + + /** + * Checks if there is at least one row left in the batch to return. + * If no more row are available, it reads another batch of rows. + * + * @return Returns true if there is one more row to return, false otherwise. + * @throws IOException throw if an exception happens while reading a batch. + */ + private boolean ensureBatch() throws IOException { + + if (nextRow >= rowsInBatch) { + // No more rows available in the Rows array. + nextRow = 0; + // Try to read the next batch if rows from the ORC file. + boolean moreRows = orcRowsReader.nextBatch(rowBatch); + + if (moreRows) { + // Load the data into the Rows array. + rowsInBatch = fillRows(rows, schema, rowBatch, selectedFields); + } + return moreRows; + } + // there is at least one Row left in the Rows array. + return true; + } + + @Override + public Row nextRecord(Row reuse) throws IOException { + // return the next row + return rows[this.nextRow++]; + } + + @Override + public TypeInformation getProducedType() { + return rowType; + } + + // -------------------------------------------------------------------------------------------- + // Custom serialization methods + // -------------------------------------------------------------------------------------------- + + private void writeObject(ObjectOutputStream out) throws IOException { + out.writeInt(batchSize); + this.conf.write(out); + out.writeUTF(schema.toString()); + + out.writeInt(selectedFields.length); + for (int f : selectedFields) { + out.writeInt(f); + } + + out.writeInt(conjunctPredicates.size()); + for (Predicate p : conjunctPredicates) { + out.writeObject(p); + } + } + + @SuppressWarnings("unchecked") + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + batchSize = in.readInt(); + org.apache.hadoop.conf.Configuration configuration = new org.apache.hadoop.conf.Configuration(); + configuration.readFields(in); + + if (this.conf == null) { + this.conf = configuration; + } + this.schema = TypeDescription.fromString(in.readUTF()); + + this.selectedFields = new int[in.readInt()]; + for (int i = 0; i < selectedFields.length; i++) { + this.selectedFields[i] = in.readInt(); + } + + this.conjunctPredicates = new ArrayList<>(); + int numPreds = in.readInt(); + for (int i = 0; i < numPreds; i++) { + conjunctPredicates.add((Predicate) in.readObject()); + } + } + + // -------------------------------------------------------------------------------------------- + // Classes to define predicates + // -------------------------------------------------------------------------------------------- + + /** + * A filter predicate that can be evaluated by the OrcRowInputFormat. + */ + public abstract static class Predicate implements Serializable { + protected abstract SearchArgument.Builder add(SearchArgument.Builder builder); + } + + abstract static class ColumnPredicate extends Predicate { + final String columnName; + final PredicateLeaf.Type literalType; + + ColumnPredicate(String columnName, PredicateLeaf.Type literalType) { + this.columnName = columnName; + this.literalType = literalType; + } + + Object castLiteral(Serializable literal) { + + switch (literalType) { + case LONG: + if (literal instanceof Byte) { + return new Long((Byte) literal); + } else if (literal instanceof Short) { + return new Long((Short) literal); + } else if (literal instanceof Integer) { + return new Long((Integer) literal); + } else if (literal instanceof Long) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a LONG column requires an integer " + + "literal, i.e., Byte, Short, Integer, or Long."); + } + case FLOAT: + if (literal instanceof Float) { + return new Double((Float) literal); + } else if (literal instanceof Double) { + return literal; + } else if (literal instanceof BigDecimal) { + return ((BigDecimal) literal).doubleValue(); + } else { + throw new IllegalArgumentException("A predicate on a FLOAT column requires a floating " + + "literal, i.e., Float or Double."); + } + case STRING: + if (literal instanceof String) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a STRING column requires a floating " + + "literal, i.e., Float or Double."); + } + case BOOLEAN: + if (literal instanceof Boolean) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a BOOLEAN column requires a Boolean literal."); + } + case DATE: + if (literal instanceof Date) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a DATE column requires a java.sql.Date literal."); + } + case TIMESTAMP: + if (literal instanceof Timestamp) { + return literal; + } else { + throw new IllegalArgumentException("A predicate on a TIMESTAMP column requires a java.sql.Timestamp literal."); + } + case DECIMAL: + if (literal instanceof BigDecimal) { + return new HiveDecimalWritable(HiveDecimal.create((BigDecimal) literal)); + } else { + throw new IllegalArgumentException("A predicate on a DECIMAL column requires a BigDecimal literal."); + } + default: + throw new IllegalArgumentException("Unknown literal type " + literalType); + } + } + } + + abstract static class BinaryPredicate extends ColumnPredicate { + final Serializable literal; + + BinaryPredicate(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType); + this.literal = literal; + } + } + + /** + * An EQUALS predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class Equals extends BinaryPredicate { + /** + * Creates an EQUALS predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literal. + * @param literal The literal value to check the column against. + */ + public Equals(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType, literal); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.equals(columnName, literalType, castLiteral(literal)); + } + + @Override + public String toString() { + return columnName + " = " + literal; + } + } + + /** + * An EQUALS predicate that can be evaluated with Null safety by the OrcRowInputFormat. + */ + public static class NullSafeEquals extends BinaryPredicate { + /** + * Creates a null-safe EQUALS predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literal. + * @param literal The literal value to check the column against. + */ + public NullSafeEquals(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType, literal); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.nullSafeEquals(columnName, literalType, castLiteral(literal)); + } + + @Override + public String toString() { + return columnName + " = " + literal; + } + } + + /** + * A LESS_THAN predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class LessThan extends BinaryPredicate { + /** + * Creates a LESS_THAN predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literal. + * @param literal The literal value to check the column against. + */ + public LessThan(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType, literal); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.lessThan(columnName, literalType, castLiteral(literal)); + } + + @Override + public String toString() { + return columnName + " < " + literal; + } + } + + /** + * A LESS_THAN_EQUALS predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class LessThanEquals extends BinaryPredicate { + /** + * Creates a LESS_THAN_EQUALS predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literal. + * @param literal The literal value to check the column against. + */ + public LessThanEquals(String columnName, PredicateLeaf.Type literalType, Serializable literal) { + super(columnName, literalType, literal); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.lessThanEquals(columnName, literalType, castLiteral(literal)); + } + + @Override + public String toString() { + return columnName + " <= " + literal; + } + } + + /** + * An IS_NULL predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class IsNull extends ColumnPredicate { + /** + * Creates an IS_NULL predicate. + * + * @param columnName The column to check for null. + * @param literalType The type of the column to check for null. + */ + public IsNull(String columnName, PredicateLeaf.Type literalType) { + super(columnName, literalType); + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.isNull(columnName, literalType); + } + + @Override + public String toString() { + return columnName + " IS NULL"; + } + } + + /** + * An BETWEEN predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class Between extends ColumnPredicate { + private Serializable lowerBound; + private Serializable upperBound; + + /** + * Creates an BETWEEN predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literals. + * @param lowerBound The literal value of the (inclusive) lower bound to check the column against. + * @param upperBound The literal value of the (inclusive) upper bound to check the column against. + */ + public Between(String columnName, PredicateLeaf.Type literalType, Serializable lowerBound, Serializable upperBound) { + super(columnName, literalType); + this.lowerBound = lowerBound; + this.upperBound = upperBound; + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return builder.between(columnName, literalType, castLiteral(lowerBound), castLiteral(upperBound)); + } + + @Override + public String toString() { + return lowerBound + " <= " + columnName + " <= " + upperBound; + } + } + + /** + * An IN predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class In extends ColumnPredicate { + private Serializable[] literals; + + /** + * Creates an IN predicate. + * + * @param columnName The column to check. + * @param literalType The type of the literals. + * @param literals The literal values to check the column against. + */ + public In(String columnName, PredicateLeaf.Type literalType, Serializable... literals) { + super(columnName, literalType); + this.literals = literals; + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + Object[] castedLiterals = new Object[literals.length]; + for (int i = 0; i < literals.length; i++) { + castedLiterals[i] = castLiteral(literals[i]); + } + return builder.in(columnName, literalType, (Object[]) castedLiterals); + } + + @Override + public String toString() { + return columnName + " IN " + Arrays.toString(literals); + } + } + + /** + * A NOT predicate to negate a predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class Not extends Predicate { + private final Predicate pred; + + /** + * Creates a NOT predicate. + * + * @param predicate The predicate to negate. + */ + public Not(Predicate predicate) { + this.pred = predicate; + } + + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + return pred.add(builder.startNot()).end(); + } + + protected Predicate child() { + return pred; + } + + @Override + public String toString() { + return "NOT(" + pred.toString() + ")"; + } + } + + /** + * An OR predicate that can be evaluated by the OrcRowInputFormat. + */ + public static class Or extends Predicate { + private final Predicate[] preds; + + /** + * Creates an OR predicate. + * + * @param predicates The disjunctive predicates. + */ + public Or(Predicate... predicates) { + this.preds = predicates; + } + + @Override + protected SearchArgument.Builder add(SearchArgument.Builder builder) { + SearchArgument.Builder withOr = builder.startOr(); + for (Predicate p : preds) { + withOr = p.add(withOr); + } + return withOr.end(); + } + + protected Iterable children() { + return Arrays.asList(preds); + } + + @Override + public String toString() { + return "OR(" + Arrays.toString(preds) + ")"; + } + } +} diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java index 0454ba45af16a..b7c537865764d 100644 --- a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java @@ -18,111 +18,474 @@ package org.apache.flink.orc; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.orc.OrcRowInputFormat.Predicate; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.expressions.Attribute; +import org.apache.flink.table.expressions.BinaryComparison; +import org.apache.flink.table.expressions.EqualTo; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.GreaterThan; +import org.apache.flink.table.expressions.GreaterThanOrEqual; +import org.apache.flink.table.expressions.IsNotNull; +import org.apache.flink.table.expressions.IsNull; +import org.apache.flink.table.expressions.LessThan; +import org.apache.flink.table.expressions.LessThanOrEqual; +import org.apache.flink.table.expressions.Literal; +import org.apache.flink.table.expressions.Not; +import org.apache.flink.table.expressions.NotEqualTo; +import org.apache.flink.table.expressions.Or; +import org.apache.flink.table.expressions.UnaryExpression; import org.apache.flink.table.sources.BatchTableSource; +import org.apache.flink.table.sources.FilterableTableSource; import org.apache.flink.table.sources.ProjectableTableSource; import org.apache.flink.table.sources.TableSource; import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; import org.apache.orc.TypeDescription; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + /** - * Creates a TableSource to read ORC file. + * A TableSource to read ORC files. * - *

The ORC file path and schema is passed during {@link OrcTableSource} construction. configuration is optional. + *

The {@link OrcTableSource} supports projection and filter push-down.

* - *

The OrcTableSource is used as shown in the example below. + *

An {@link OrcTableSource} is used as shown in the example below. * *

  * {@code
- * String path = testInputURL.getPath();
- * String schema = "struct"
- * OrcTableSource orcSrc = new OrcTableSource(path, schema);
+ * OrcTableSource orcSrc = OrcTableSource.builder()
+ *   .path("file:///my/data/file.orc")
+ *   .forOrcSchema("struct")
+ *   .build();
+ *
  * tEnv.registerTableSource("orcTable", orcSrc);
  * Table res = tableEnv.sql("SELECT * FROM orcTable");
  * }
  * 
*/ -public class OrcTableSource implements BatchTableSource, ProjectableTableSource { +public class OrcTableSource + implements BatchTableSource, ProjectableTableSource, FilterableTableSource { - private String path; - private TypeDescription orcSchema; - private RowTypeInfo typeInfo; - private Configuration orcConfig; - private int[] fieldMapping; + private static final int DEFAULT_BATCH_SIZE = 1000; - /** - * The ORC file path and schema. - * - * @param path the path of orc file - * @param orcSchema schema of orc file - */ - public OrcTableSource(String path, String orcSchema) { - this(path, orcSchema, new Configuration()); - } + // path to read ORC files from + private final String path; + // schema of the ORC file + private final TypeDescription orcSchema; + // the schema of the Table + private final TableSchema tableSchema; + // the configuration to read the file + private final Configuration orcConfig; + // the number of rows to read in a batch + private final int batchSize; + + // type information of the data returned by the InputFormat + private final RowTypeInfo typeInfo; + // list of selected ORC fields to return + private final int[] selectedFields; + // list of predicates to apply + private final Predicate[] predicates; /** - * The file path and schema of orc file, and configuration to read orc file . + * Creates an OrcTableSouce from an ORC TypeDescription. * - * @param path the path of orc file - * @param orcSchema schema of orc file - * @param orcConfig configuration to read orc file + * @param path The path to read the ORC files from. + * @param orcSchema The schema of the ORC files as TypeDescription. + * @param orcConfig The configuration to read the ORC files. + * @param batchSize The number of Rows to read in a batch, default is 1000. */ - public OrcTableSource(String path, String orcSchema, Configuration orcConfig) { - this(path, TypeDescription.fromString(orcSchema), orcConfig); + private OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig, int batchSize) { + this(path, orcSchema, orcConfig, batchSize, null, null); } - public OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig) { + private OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig, + int batchSize, int[] selectedFields, Predicate[] predicates) { + + Preconditions.checkNotNull(path, "Path must not be null."); + Preconditions.checkNotNull(orcSchema, "OrcSchema must not be null."); + Preconditions.checkNotNull(path, "Configuration must not be null."); + Preconditions.checkArgument(batchSize > 0, "Batch size must be larger than null."); this.path = path; this.orcSchema = orcSchema; this.orcConfig = orcConfig; + this.batchSize = batchSize; + this.selectedFields = selectedFields; + this.predicates = predicates; - this.typeInfo = (RowTypeInfo) OrcUtils.schemaToTypeInfo(this.orcSchema); + // determine the type information from the ORC schema + RowTypeInfo typeInfoFromSchema = (RowTypeInfo) OrcUtils.schemaToTypeInfo(this.orcSchema); + + // set return type info + if (selectedFields == null) { + this.typeInfo = typeInfoFromSchema; + } else { + this.typeInfo = RowTypeInfo.projectFields(typeInfoFromSchema, selectedFields); + } + // create a TableSchema that corresponds to the ORC schema + this.tableSchema = new TableSchema( + typeInfoFromSchema.getFieldNames(), + typeInfoFromSchema.getFieldTypes() + ); } @Override public DataSet getDataSet(ExecutionEnvironment execEnv) { - - RowOrcInputFormat orcIF = new RowOrcInputFormat(path, orcSchema, orcConfig); - if (fieldMapping != null) { - orcIF.setFieldMapping(fieldMapping); + OrcRowInputFormat orcIF = buildOrcInputFormat(); + if (selectedFields != null) { + orcIF.selectFields(selectedFields); + } + if (predicates != null) { + for (OrcRowInputFormat.Predicate pred : predicates) { + orcIF.addPredicate(pred); + } } return execEnv.createInput(orcIF); } + @VisibleForTesting + protected OrcRowInputFormat buildOrcInputFormat() { + return new OrcRowInputFormat(path, orcSchema, orcConfig, batchSize); + } + @Override public TypeInformation getReturnType() { return typeInfo; } @Override - public TableSource projectFields(int[] fields) { + public TableSchema getTableSchema() { + return this.tableSchema; + } - OrcTableSource copy = new OrcTableSource(path, orcSchema, orcConfig); + @Override + public TableSource projectFields(int[] selectedFields) { + // create a copy of the OrcTableSouce with new selected fields + return new OrcTableSource(path, orcSchema, orcConfig, batchSize, selectedFields, predicates); + } - // set field mapping - copy.fieldMapping = fields; + @Override + public TableSource applyPredicate(List predicates) { + ArrayList orcPredicates = new ArrayList<>(); - // adapt TypeInfo - TypeInformation[] fieldTypes = new TypeInformation[fields.length]; - String[] fieldNames = new String[fields.length]; - for (int i = 0; i < fields.length; i++) { - fieldTypes[i] = this.typeInfo.getTypeAt(fields[i]); - fieldNames[i] = this.typeInfo.getFieldNames()[fields[i]]; + // we do not remove any predicates from the list because ORC does not fully apply predicates + for (Expression pred : predicates) { + Predicate orcPred = toOrcPredicate(pred); + if (orcPred != null) { + orcPredicates.add(orcPred); + } } - copy.typeInfo = new RowTypeInfo(fieldTypes, fieldNames); - return copy; + return new OrcTableSource(path, orcSchema, orcConfig, batchSize, selectedFields, orcPredicates.toArray(new Predicate[]{})); + } + + @Override + public boolean isFilterPushedDown() { + return this.predicates != null; } @Override public String explainSource() { - return "ORC Source file at path " + this.path + " with schema " + this.orcSchema; + return "OrcFile[path=" + path + ", schema=" + orcSchema + ", filter=" + predicateString() + "]"; + } + + private String predicateString() { + if (predicates != null) { + return "AND(" + Arrays.toString(predicates) + ")"; + } else { + return "TRUE"; + } + } + + // Predicate conversion for filter push-down. + + private Predicate toOrcPredicate(Expression pred) { + if (pred instanceof Or) { + Predicate c1 = toOrcPredicate(((Or) pred).left()); + Predicate c2 = toOrcPredicate(((Or) pred).right()); + if (c1 == null || c2 == null) { + return null; + } else { + return new OrcRowInputFormat.Or(c1, c2); + } + } else if (pred instanceof Not) { + Predicate c = toOrcPredicate(((Not) pred).child()); + if (c == null) { + return null; + } else { + return new OrcRowInputFormat.Not(c); + } + } else if (pred instanceof BinaryComparison) { + + BinaryComparison binComp = (BinaryComparison) pred; + + if (!isValid(binComp)) { + // not a valid predicate + return null; + } + PredicateLeaf.Type litType = getLiteralType(binComp); + if (litType == null) { + // unsupported literal type + return null; + } + + boolean literalOnRight = literalOnRight(binComp); + String colName = getColumnName(binComp); + Serializable literal = (Serializable) getLiteral(binComp); + + if (pred instanceof EqualTo) { + return new OrcRowInputFormat.Equals(colName, litType, literal); + } else if (pred instanceof NotEqualTo) { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.Equals(colName, litType, literal)); + } else if (pred instanceof GreaterThan) { + if (literalOnRight) { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThanEquals(colName, litType, literal)); + } else { + return new OrcRowInputFormat.LessThan(colName, litType, literal); + } + } else if (pred instanceof GreaterThanOrEqual) { + if (literalOnRight) { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThan(colName, litType, literal)); + } else { + return new OrcRowInputFormat.LessThanEquals(colName, litType, literal); + } + } else if (pred instanceof LessThan) { + if (literalOnRight) { + return new OrcRowInputFormat.LessThan(colName, litType, literal); + } else { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThanEquals(colName, litType, literal)); + } + } else if (pred instanceof LessThanOrEqual) { + if (literalOnRight) { + return new OrcRowInputFormat.LessThanEquals(colName, litType, literal); + } else { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThan(colName, litType, literal)); + } + } else { + // unsupported predicate + return null; + } + } else if (pred instanceof UnaryExpression) { + + UnaryExpression unary = (UnaryExpression) pred; + if (!isValid(unary)) { + // not a valid predicate + return null; + } + PredicateLeaf.Type colType = toOrcType(((UnaryExpression) pred).child().resultType()); + if (colType == null) { + // unsupported type + return null; + } + + String colName = getColumnName(unary); + + if (pred instanceof IsNull) { + return new OrcRowInputFormat.IsNull(colName, colType); + } else if (pred instanceof IsNotNull) { + return new OrcRowInputFormat.Not( + new OrcRowInputFormat.IsNull(colName, colType)); + } else { + // unsupported predicate + return null; + } + } else { + // unsupported predicate + return null; + } + } + + private boolean isValid(UnaryExpression unary) { + return unary.child() instanceof Attribute; + } + + private boolean isValid(BinaryComparison comp) { + return (comp.left() instanceof Literal && comp.right() instanceof Attribute) || + (comp.left() instanceof Attribute && comp.right() instanceof Literal); + } + + private boolean literalOnRight(BinaryComparison comp) { + if (comp.left() instanceof Literal && comp.right() instanceof Attribute) { + return false; + } else if (comp.left() instanceof Attribute && comp.right() instanceof Literal) { + return true; + } else { + throw new RuntimeException("Invalid binary comparison."); + } + } + + private String getColumnName(UnaryExpression unary) { + return ((Attribute) unary.child()).name(); + } + + private String getColumnName(BinaryComparison comp) { + if (literalOnRight(comp)) { + return ((Attribute) comp.left()).name(); + } else { + return ((Attribute) comp.right()).name(); + } + } + + private PredicateLeaf.Type getLiteralType(BinaryComparison comp) { + if (literalOnRight(comp)) { + return toOrcType(((Literal) comp.right()).resultType()); + } else { + return toOrcType(((Literal) comp.left()).resultType()); + } + } + + private Object getLiteral(BinaryComparison comp) { + if (literalOnRight(comp)) { + return ((Literal) comp.right()).value(); + } else { + return ((Literal) comp.left()).value(); + } + } + + private PredicateLeaf.Type toOrcType(TypeInformation type) { + if (type == BasicTypeInfo.BYTE_TYPE_INFO || + type == BasicTypeInfo.SHORT_TYPE_INFO || + type == BasicTypeInfo.INT_TYPE_INFO || + type == BasicTypeInfo.LONG_TYPE_INFO) { + return PredicateLeaf.Type.LONG; + } else if (type == BasicTypeInfo.FLOAT_TYPE_INFO || + type == BasicTypeInfo.DOUBLE_TYPE_INFO) { + return PredicateLeaf.Type.FLOAT; + } else if (type == BasicTypeInfo.BOOLEAN_TYPE_INFO) { + return PredicateLeaf.Type.BOOLEAN; + } else if (type == BasicTypeInfo.STRING_TYPE_INFO) { + return PredicateLeaf.Type.STRING; + } else if (type == SqlTimeTypeInfo.TIMESTAMP) { + return PredicateLeaf.Type.TIMESTAMP; + } else if (type == SqlTimeTypeInfo.DATE) { + return PredicateLeaf.Type.DATE; + } else if (type == BasicTypeInfo.BIG_DEC_TYPE_INFO) { + return PredicateLeaf.Type.DECIMAL; + } else { + // unsupported type + return null; + } + } + + // Builder + + public static Builder builder() { + return new Builder(); + } + + /** + * Constructs an {@link OrcTableSource}. + */ + public static class Builder { + + private String path; + + private TypeDescription schema; + + private Configuration config; + + private int batchSize = 0; + + /** + * Sets the path of the ORC file(s). + * + * @param path The path of the ORC file(s). + * @return The builder. + */ + public Builder path(String path) { + Preconditions.checkNotNull(path, "Path must not be null."); + Preconditions.checkArgument(!path.isEmpty(), "Path must not be empty."); + this.path = path; + return this; + } + + /** + * Sets the ORC schema of the files to read as a String. + * + * @param orcSchema The ORC schema of the files to read as a String. + * @return The builder. + */ + public Builder forOrcSchema(String orcSchema) { + Preconditions.checkNotNull(orcSchema, "ORC schema must not be null."); + this.schema = TypeDescription.fromString(orcSchema); + return this; + } + + /** + * Sets the ORC schema of the files to read as a {@link TypeDescription}. + * + * @param orcSchema The ORC schema of the files to read as a String. + * @return The builder. + */ + public Builder forOrcSchema(TypeDescription orcSchema) { + Preconditions.checkNotNull(orcSchema, "ORC Schema must not be null."); + this.schema = orcSchema; + return this; + } + + /** + * Sets a Hadoop {@link Configuration} for the ORC reader. If no configuration is configured, + * an empty configuration is used. + * + * @param config The Hadoop Configuration for the ORC reader. + * @return The builder. + */ + public Builder withConfiguration(Configuration config) { + Preconditions.checkNotNull(config, "Configuration must not be null."); + this.config = config; + return this; + } + + /** + * Sets the number of rows that are read in a batch. If not configured, the ORC files are + * read with a batch size of 1000. + * + * @param batchSize The number of rows that are read in a batch. + * @return The builder. + */ + public Builder withBatchSize(int batchSize) { + Preconditions.checkArgument(batchSize > 0, "Batch size must be greater than zero."); + this.batchSize = batchSize; + return this; + } + + /** + * Builds the OrcTableSource for this builder. + * + * @return The OrcTableSource for this builder. + */ + public OrcTableSource build() { + Preconditions.checkNotNull(this.path, "Path must not be null."); + Preconditions.checkNotNull(this.schema, "ORC schema must not be null."); + if (this.config == null) { + this.config = new Configuration(); + } + if (this.batchSize == 0) { + // set default batch size + this.batchSize = DEFAULT_BATCH_SIZE; + } + return new OrcTableSource(this.path, this.schema, this.config, this.batchSize); + } + } } diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java index c7557c7477be1..cfb4e0e66a818 100644 --- a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcUtils.java @@ -39,29 +39,36 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; - import org.apache.orc.TypeDescription; import java.lang.reflect.Array; import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; import java.sql.Date; import java.sql.Timestamp; import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.TimeZone; +import java.util.function.DoubleFunction; +import java.util.function.IntFunction; +import java.util.function.LongFunction; /** * A class that provides utility methods for orc file reading. */ -public class OrcUtils { +class OrcUtils { + + private static final long MILLIS_PER_DAY = 86400000; // = 24 * 60 * 60 * 1000 + private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); /** - * Convert ORC schema types to Flink types. - * - * @param schema schema of orc file + * Converts an ORC schema to a Flink TypeInformation. * + * @param schema The ORC schema. + * @return The TypeInformation that corresponds to the ORC schema. */ - public static TypeInformation schemaToTypeInfo(TypeDescription schema) { + static TypeInformation schemaToTypeInfo(TypeDescription schema) { switch (schema.getCategory()) { case BOOLEAN: return BasicTypeInfo.BOOLEAN_TYPE_INFO; @@ -77,6 +84,8 @@ public static TypeInformation schemaToTypeInfo(TypeDescription schema) { return BasicTypeInfo.FLOAT_TYPE_INFO; case DOUBLE: return BasicTypeInfo.DOUBLE_TYPE_INFO; + case DECIMAL: + return BasicTypeInfo.BIG_DEC_TYPE_INFO; case STRING: case CHAR: case VARCHAR: @@ -97,154 +106,164 @@ public static TypeInformation schemaToTypeInfo(TypeDescription schema) { return new RowTypeInfo(fieldTypes, fieldNames); case LIST: TypeDescription elementSchema = schema.getChildren().get(0); - TypeInformation elementType = schemaToTypeInfo(elementSchema); + TypeInformation elementType = schemaToTypeInfo(elementSchema); + // arrays of primitive types are handled as object arrays to support null values return ObjectArrayTypeInfo.getInfoFor(elementType); case MAP: TypeDescription keySchema = schema.getChildren().get(0); TypeDescription valSchema = schema.getChildren().get(1); - TypeInformation keyType = schemaToTypeInfo(keySchema); - TypeInformation valType = schemaToTypeInfo(valSchema); - return new MapTypeInfo(keyType, valType); - case DECIMAL: - return BasicTypeInfo.BIG_DEC_TYPE_INFO; + TypeInformation keyType = schemaToTypeInfo(keySchema); + TypeInformation valType = schemaToTypeInfo(valSchema); + return new MapTypeInfo<>(keyType, valType); case UNION: - throw new UnsupportedOperationException("UNION type not supported yet."); + throw new UnsupportedOperationException("UNION type is not supported yet."); default: throw new IllegalArgumentException("Unknown type " + schema); } } /** - * Fill rows from orc batch. - * - * @param rows the batch of rows need to be filled - * @param schema schema of orc file - * @param batch current orc batch data used to fill the rows - * @param fieldMapping field mapping + * Fills an ORC batch into an array of Row. * + * @param rows The batch of rows need to be filled. + * @param schema The schema of the ORC data. + * @param batch The ORC data. + * @param selectedFields The list of selected ORC fields. + * @return The number of rows that were filled. */ - public static void fillRows(Object[] rows, TypeDescription schema, VectorizedRowBatch batch, int[] fieldMapping) { + static int fillRows(Row[] rows, TypeDescription schema, VectorizedRowBatch batch, int[] selectedFields) { - int totalRowsInBatch = (int) batch.count(); + int rowsToRead = Math.min((int) batch.count(), rows.length); List fieldTypes = schema.getChildren(); - for (int outIdx = 0; outIdx < fieldMapping.length; outIdx++) { - int inIdx = fieldMapping[outIdx]; - readField(rows, outIdx, fieldTypes.get(inIdx), batch.cols[inIdx], null, Math.min((int) totalRowsInBatch, rows.length)); + // read each selected field + for (int rowIdx = 0; rowIdx < selectedFields.length; rowIdx++) { + int orcIdx = selectedFields[rowIdx]; + readField(rows, rowIdx, fieldTypes.get(orcIdx), batch.cols[orcIdx], null, rowsToRead); } + return rowsToRead; } - private static void readField(Object[] rows, int fieldIdx, TypeDescription schema, ColumnVector vector, long[] lengthVector, int childCount) { + /** + * Reads a vector of data into an array of objects. + * + * @param vals The array that needs to be filled. + * @param fieldIdx If the vals array is an array of Row, the index of the field that needs to be filled. + * Otherwise a -1 must be passed and the data is directly filled into the array. + * @param schema The schema of the vector to read. + * @param vector The vector to read. + * @param lengthVector If the vector is of type List or Map, the number of sub-elements to read for each field. Otherwise, it must be null. + * @param childCount The number of vector entries to read. + */ + private static void readField(Object[] vals, int fieldIdx, TypeDescription schema, ColumnVector vector, long[] lengthVector, int childCount) { + // check the type of the vector to decide how to read it. switch (schema.getCategory()) { case BOOLEAN: if (vector.noNulls) { - readNonNullBooleanColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readBoolean, OrcUtils::boolArray); } else { - readBooleanColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readBoolean, OrcUtils::boolArray); } break; case BYTE: if (vector.noNulls) { - readNonNullByteColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readByte, OrcUtils::byteArray); } else { - readByteColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readByte, OrcUtils::byteArray); } break; case SHORT: if (vector.noNulls) { - readNonNullShortColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readShort, OrcUtils::shortArray); } else { - readShortColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readShort, OrcUtils::shortArray); } break; case INT: if (vector.noNulls) { - readNonNullIntColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readInt, OrcUtils::intArray); } else { - readIntColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readInt, OrcUtils::intArray); } break; case LONG: if (vector.noNulls) { - readNonNullLongColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readLong, OrcUtils::longArray); } else { - readLongColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumn(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount, OrcUtils::readLong, OrcUtils::longArray); } break; case FLOAT: if (vector.noNulls) { - readNonNullFloatColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + readNonNullDoubleColumn(vals, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount, OrcUtils::readFloat, OrcUtils::floatArray); } else { - readFloatColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + readDoubleColumn(vals, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount, OrcUtils::readFloat, OrcUtils::floatArray); } break; case DOUBLE: if (vector.noNulls) { - readNonNullDoubleColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + readNonNullDoubleColumn(vals, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount, OrcUtils::readDouble, OrcUtils::doubleArray); } else { - readDoubleColumn(rows, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount); + readDoubleColumn(vals, fieldIdx, (DoubleColumnVector) vector, lengthVector, childCount, OrcUtils::readDouble, OrcUtils::doubleArray); } break; case CHAR: case VARCHAR: case STRING: if (vector.noNulls) { - readNonNullStringColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + readNonNullBytesColumnAsString(vals, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); } else { - readStringColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + readBytesColumnAsString(vals, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); } break; case DATE: if (vector.noNulls) { - readNonNullDateColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readNonNullLongColumnAsDate(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); } else { - readDateColumn(rows, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); + readLongColumnAsDate(vals, fieldIdx, (LongColumnVector) vector, lengthVector, childCount); } break; case TIMESTAMP: if (vector.noNulls) { - readNonNullTimestampColumn(rows, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); + readNonNullTimestampColumn(vals, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); } else { - readTimestampColumn(rows, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); + readTimestampColumn(vals, fieldIdx, (TimestampColumnVector) vector, lengthVector, childCount); } break; case BINARY: if (vector.noNulls) { - readNonNullBinaryColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + readNonNullBytesColumnAsBinary(vals, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); } else { - readBinaryColumn(rows, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); + readBytesColumnAsBinary(vals, fieldIdx, (BytesColumnVector) vector, lengthVector, childCount); } break; case DECIMAL: if (vector.noNulls) { - readNonNullDecimalColumn(rows, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); - } - else { - readDecimalColumn(rows, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); + readNonNullDecimalColumn(vals, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); + } else { + readDecimalColumn(vals, fieldIdx, (DecimalColumnVector) vector, lengthVector, childCount); } break; case STRUCT: if (vector.noNulls) { - readNonNullStructColumn(rows, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); + readNonNullStructColumn(vals, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); } else { - readStructColumn(rows, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); + readStructColumn(vals, fieldIdx, (StructColumnVector) vector, schema, lengthVector, childCount); } break; case LIST: if (vector.noNulls) { - readNonNullListColumn(rows, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); - } - else { - readListColumn(rows, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); + readNonNullListColumn(vals, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); + } else { + readListColumn(vals, fieldIdx, (ListColumnVector) vector, schema, lengthVector, childCount); } break; case MAP: if (vector.noNulls) { - readNonNullMapColumn(rows, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); - } - else { - readMapColumn(rows, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); + readNonNullMapColumn(vals, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); + } else { + readMapColumn(vals, fieldIdx, (MapColumnVector) vector, schema, lengthVector, childCount); } break; case UNION: @@ -254,1870 +273,1013 @@ private static void readField(Object[] rows, int fieldIdx, TypeDescription schem } } - private static void readNonNullBooleanColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullLongColumn(Object[] vals, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount, + LongFunction reader, IntFunction array) { - // check if boolean is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - boolean repeatingValue = vector.vector[0] != 0; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + T repeatingValue = reader.apply(vector.vector[0]); + fillColumnWithRepeatingValue(vals, fieldIdx, repeatingValue, childCount); } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = vector.vector[i] != 0; + vals[i] = reader.apply(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, vector.vector[i] != 0); + rows[i].setField(fieldIdx, reader.apply(vector.vector[i])); } } } } else { // in a list - boolean[] temp; + T[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - boolean repeatingValue = vector.vector[0] != 0; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + T repeatingValue = reader.apply(vector.vector[0]); + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + Arrays.fill(temp, repeatingValue); + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++] != 0; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + for (int j = 0; j < temp.length; j++) { + temp[j] = reader.apply(vector.vector[offset++]); } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++] != 0; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullByteColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullDoubleColumn(Object[] vals, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount, + DoubleFunction reader, IntFunction array) { - // check if byte is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - byte repeatingValue = (byte) vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + T repeatingValue = reader.apply(vector.vector[0]); + fillColumnWithRepeatingValue(vals, fieldIdx, repeatingValue, childCount); } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = (byte) vector.vector[i]; + vals[i] = reader.apply(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (byte) vector.vector[i]); + rows[i].setField(fieldIdx, reader.apply(vector.vector[i])); } } } } else { // in a list - byte[] temp; + T[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - byte repeatingValue = (byte) vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + T repeatingValue = reader.apply(vector.vector[0]); + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + Arrays.fill(temp, repeatingValue); + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (byte) vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + for (int j = 0; j < temp.length; j++) { + temp[j] = reader.apply(vector.vector[offset++]); } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (byte) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullShortColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if short is directly in a list or not, e.g, array + private static void readNonNullBytesColumnAsString(Object[] vals, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + // check if the values need to be read into lists or as single values if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - short repeatingValue = (short) vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + if (bytes.isRepeating) { // fill complete column with first value + String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); + fillColumnWithRepeatingValue(vals, fieldIdx, repeatingValue, childCount); } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = (short) vector.vector[i]; + vals[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i], StandardCharsets.UTF_8); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (short) vector.vector[i]); + rows[i].setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i], StandardCharsets.UTF_8)); } } } - } else { // in a list - short[] temp; + } else { + String[] temp; int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - short repeatingValue = (short) vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + if (bytes.isRepeating) { // fill complete list with first value + String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0], StandardCharsets.UTF_8); + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (short) vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset], StandardCharsets.UTF_8); + offset++; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (short) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullIntColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if int is directly in a list or not, e.g, array + private static void readNonNullBytesColumnAsBinary(Object[] vals, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + // check if the values need to be read into lists or as single values if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - int repeatingValue = (int) vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + if (bytes.isRepeating) { // fill complete column with first value + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + // don't reuse repeating val to avoid object mutation + vals[i] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + } else { // set as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + // don't reuse repeating val to avoid object mutation + rows[i].setField(fieldIdx, readBinary(bytes.vector[0], bytes.start[0], bytes.length[0])); + } + } } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = (int) vector.vector[i]; + vals[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (int) vector.vector[i]); + rows[i].setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); } } } - } else { // in a list - int[] temp; + } else { + byte[][] temp; int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - int repeatingValue = (int) vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + if (bytes.isRepeating) { // fill complete list with first value + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); + } + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (int) vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (int) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullLongColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullLongColumnAsDate(Object[] vals, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - // check if long is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - long repeatingValue = vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + // do not reuse repeated value due to mutability of Date + vals[i] = readDate(vector.vector[0]); + } + } else { // set as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + // do not reuse repeated value due to mutability of Date + rows[i].setField(fieldIdx, readDate(vector.vector[0])); + } + } } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = vector.vector[i]; + vals[i] = readDate(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (Long) vector.vector[i]); + rows[i].setField(fieldIdx, readDate(vector.vector[i])); } } } } else { // in a list - long[] temp; + Date[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - long repeatingValue = vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[0]); + } + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readDate(vector.vector[offset++]); } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullFloatColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullTimestampColumn(Object[] vals, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { - // check if float is directly in a list or not, e.g, array + // check if the timestamps need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - float repeatingValue = (float) vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + if (fieldIdx == -1) { // set as an object + for (int i = 0; i < childCount; i++) { + // do not reuse value to prevent object mutation + vals[i] = readTimestamp(vector.time[0], vector.nanos[0]); + } + } else { // set as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + // do not reuse value to prevent object mutation + rows[i].setField(fieldIdx, readTimestamp(vector.time[0], vector.nanos[0])); + } + } } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = (float) vector.vector[i]; + vals[i] = readTimestamp(vector.time[i], vector.nanos[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, (float) vector.vector[i]); + rows[i].setField(fieldIdx, readTimestamp(vector.time[i], vector.nanos[i])); } } } - } else { // in a list - float[] temp; + } else { + Timestamp[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - float repeatingValue = (float) vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + // do not reuse value to prevent object mutation + temp[j] = readTimestamp(vector.time[0], vector.nanos[0]); + } + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (float) vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readTimestamp(vector.time[offset], vector.nanos[offset]); + offset++; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = (float) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullDoubleColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { + private static void readNonNullDecimalColumn(Object[] vals, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { - // check if double is directly in a list or not, e.g, array + // check if the decimals need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - double repeatingValue = vector.vector[0]; - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); + fillColumnWithRepeatingValue(vals, fieldIdx, readBigDecimal(vector.vector[0]), childCount); } else { if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = vector.vector[i]; + vals[i] = readBigDecimal(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); + rows[i].setField(fieldIdx, readBigDecimal(vector.vector[i])); } } } - } else { // in a list - double[] temp; + } else { + BigDecimal[] temp; int offset = 0; if (vector.isRepeating) { // fill complete list with first value - double repeatingValue = vector.vector[0]; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; + BigDecimal repeatingValue = readBigDecimal(vector.vector[0]); + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + Arrays.fill(temp, repeatingValue); + offset += temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++]; - } - rows[i] = temp; + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + temp[j] = readBigDecimal(vector.vector[offset++]); } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } + } - private static void readNonNullStringColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + private static void readNonNullStructColumn(Object[] vals, int fieldIdx, StructColumnVector structVector, TypeDescription schema, long[] lengthVector, int childCount) { + + List childrenTypes = schema.getChildren(); + + int numFields = childrenTypes.size(); + // create a batch of Rows to read the structs + Row[] structs = new Row[childCount]; + // TODO: possible improvement: reuse existing Row objects + for (int i = 0; i < childCount; i++) { + structs[i] = new Row(numFields); + } + + // read struct fields + for (int i = 0; i < numFields; i++) { + readField(structs, i, childrenTypes.get(i), structVector.fields[i], null, childCount); + } - // check if string is directly in a list or not, e.g, array + // check if the structs need to be read into lists or as single values if (lengthVector == null) { - if (bytes.isRepeating) { // fill complete column with first value - String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); - fillColumnWithRepeatingValue(rows, fieldIdx, repeatingValue, childCount); - } else { - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); - } + if (fieldIdx == -1) { // set struct as an object + System.arraycopy(structs, 0, vals, 0, childCount); + } else { // set struct as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + rows[i].setField(fieldIdx, structs[i]); } } - } - else { // in a list - String[] temp; + } else { // struct in a list int offset = 0; - if (bytes.isRepeating) { // fill list with first value - String repeatingValue = new String(bytes.vector[0], bytes.start[0], bytes.length[0]); - if (fieldIdx == -1) { // set list as an object - for (int i = 0; i < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field - for (int i = 0; i < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; - } + Row[] temp; + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + System.arraycopy(structs, offset, temp, 0, temp.length); + offset = offset + temp.length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } - } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - rows[i] = temp; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } + } + } + } + + private static void readNonNullListColumn(Object[] vals, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { + + TypeDescription fieldType = schema.getChildren().get(0); + // check if the list need to be read into lists or as single values + if (lengthVector == null) { + long[] lengthVectorNested = list.lengths; + readField(vals, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); + } else { // list in a list + Object[] nestedLists = new Object[childCount]; + // length vector for nested list + long[] lengthVectorNested = list.lengths; + // read nested list + readField(nestedLists, -1, fieldType, list.child, lengthVectorNested, list.childCount); + // get type of nestedList + Class classType = nestedLists[0].getClass(); + + // fill outer list with nested list + int offset = 0; + int length; + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object[] temp = (Object[]) Array.newInstance(classType, length); + System.arraycopy(nestedLists, offset, temp, 0, length); + offset = offset + length; + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } + } + + private static void readNonNullMapColumn(Object[] vals, int fieldIdx, MapColumnVector mapsVector, TypeDescription schema, long[] lengthVector, int childCount) { + + List fieldType = schema.getChildren(); + TypeDescription keyType = fieldType.get(0); + TypeDescription valueType = fieldType.get(1); + + ColumnVector keys = mapsVector.keys; + ColumnVector values = mapsVector.values; + Object[] keyRows = new Object[mapsVector.childCount]; + Object[] valueRows = new Object[mapsVector.childCount]; + + // read map keys and values + readField(keyRows, -1, keyType, keys, null, keyRows.length); + readField(valueRows, -1, valueType, values, null, valueRows.length); + + // check if the maps need to be read into lists or as single values + if (lengthVector == null) { + long[] lengthVectorMap = mapsVector.lengths; + int offset = 0; + + for (int i = 0; i < childCount; i++) { + long numMapEntries = lengthVectorMap[i]; + HashMap map = readHashMap(keyRows, valueRows, offset, numMapEntries); + offset += numMapEntries; + + if (fieldIdx == -1) { + vals[i] = map; + } else { + ((Row) vals[i]).setField(fieldIdx, map); + } + } + } else { // list of map + + long[] lengthVectorMap = mapsVector.lengths; + int mapOffset = 0; // offset of map element + int offset = 0; // offset of map + HashMap[] temp; + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + long numMapEntries = lengthVectorMap[offset]; + temp[j] = readHashMap(keyRows, valueRows, mapOffset, numMapEntries); + mapOffset += numMapEntries; + offset++; + } + if (fieldIdx == 1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); + } + } + } } - private static void readNonNullDateColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readLongColumn(Object[] vals, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount, + LongFunction reader, IntFunction array) { - // check if date is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readDate(vector.vector[0]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[0])); - } - } + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { + boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = readDate(vector.vector[i]); + if (isNullVector[i]) { + vals[i] = null; + } else { + vals[i] = reader.apply(vector.vector[i]); + } } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[i])); - } - } - } - } else { - Date[] temp; - int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readDate(vector.vector[0]); - } - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readDate(vector.vector[0]); + if (isNullVector[i]) { + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, reader.apply(vector.vector[i])); } - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; } } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, array); } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readDate(vector.vector[offset++]); + // column contain null values + int offset = 0; + T[] temp; + boolean[] isNullVector = vector.isNull; + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + } else { + temp[j] = reader.apply(vector.vector[offset++]); } - rows[i] = temp; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readDate(vector.vector[offset++]); - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullTimestampColumn(Object[] rows, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { + private static void readDoubleColumn(Object[] vals, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount, + DoubleFunction reader, IntFunction array) { - // check if timestamp is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readTimeStamp(vector.time[0], vector.nanos[0]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readTimeStamp(vector.time[0], vector.nanos[0])); - } - } + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { + boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { - rows[i] = readTimeStamp(vector.time[i], vector.nanos[i]); + if (isNullVector[i]) { + vals[i] = null; + } else { + vals[i] = reader.apply(vector.vector[i]); + } } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readTimeStamp(vector.time[i], vector.nanos[i])); - } - } - } - } else { - Timestamp[] temp; - int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readTimeStamp(vector.time[0], vector.nanos[0]); - } - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readTimeStamp(vector.time[0], vector.nanos[0]); + if (isNullVector[i]) { + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, reader.apply(vector.vector[i])); } - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; } } + } + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, array); } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readTimeStamp(vector.time[offset], vector.nanos[offset]); + // column contain null values + int offset = 0; + T[] temp; + boolean[] isNullVector = vector.isNull; + for (int i = 0; offset < childCount; i++) { + temp = array.apply((int) lengthVector[i]); + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { offset++; + } else { + temp[j] = reader.apply(vector.vector[offset++]); } - rows[i] = temp; } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readTimeStamp(vector.time[offset], vector.nanos[offset]); - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readNonNullBinaryColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + private static void readBytesColumnAsString(Object[] vals, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { - // check if string is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (bytes.isRepeating) { // fill complete column with first value - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[0], bytes.start[0], bytes.length[0])); - } - } - } else { - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); - } - } - } - } else { - byte[][] temp; - int offset = 0; - if (bytes.isRepeating) { // fill complete list with first value - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); - } - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBinary(bytes.vector[0], bytes.start[0], bytes.length[0]); - } - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; - } - } - } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - rows[i] = temp; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - - } - - private static void readNonNullDecimalColumn(Object[] rows, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { - - // check if decimal is directly in a list or not, e.g, array - if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - fillColumnWithRepeatingValue(rows, fieldIdx, readBigDecimal(vector.vector[0]), childCount); - } else { - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = readBigDecimal(vector.vector[i]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readBigDecimal(vector.vector[i])); - } - } - } - } else { - BigDecimal[] temp; - int offset = 0; - if (vector.isRepeating) { // fill complete list with first value - BigDecimal repeatingValue = readBigDecimal(vector.vector[0]); - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - rows[i] = temp; - offset += temp.length; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - Arrays.fill(temp, repeatingValue); - ((Row) rows[i]).setField(fieldIdx, temp); - offset += temp.length; - } - } - } else { - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBigDecimal(vector.vector[offset++]); - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readBigDecimal(vector.vector[offset++]); - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - - } - - private static void readNonNullStructColumn(Object[] rows, int fieldIdx, StructColumnVector struct, TypeDescription schema, long[] lengthVector, int childCount) { - - List childrenTypes = schema.getChildren(); - - int numChildren = childrenTypes.size(); - Row[] nestedFields = new Row[childCount]; - for (int i = 0; i < childCount; i++) { - nestedFields[i] = new Row(numChildren); - } - for (int i = 0; i < numChildren; i++) { - readField(nestedFields, i, childrenTypes.get(i), struct.fields[i], null, childCount); - } - - // check if struct is directly in a list or not, e.g, array> - if (lengthVector == null) { - if (fieldIdx == -1) { // set struct as an object - System.arraycopy(nestedFields, 0, rows, 0, childCount); - } - else { // set struct as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, nestedFields[i]); - } - } - } - else { // struct in a list - int offset = 0; - Row[] temp; - if (fieldIdx == -1) { // set list of struct as an object - for (int i = 0; offset < childCount; i++) { - temp = new Row[(int) lengthVector[i]]; - System.arraycopy(nestedFields, offset, temp, 0, temp.length); - offset = offset + temp.length; - rows[i] = temp; - } - } - else { // set list of struct as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Row[(int) lengthVector[i]]; - System.arraycopy(nestedFields, offset, temp, 0, temp.length); - offset = offset + temp.length; - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - - private static void readNonNullListColumn(Object[] rows, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { - - TypeDescription fieldType = schema.getChildren().get(0); - if (lengthVector == null) { - long[] lengthVectorNested = list.lengths; - readField(rows, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); - } - else { // list in a list - - Object[] nestedList = new Object[childCount]; - - // length vector for nested list - long[] lengthVectorNested = list.lengths; - - // read nested list - readField(nestedList, -1, fieldType, list.child, lengthVectorNested, list.childCount); - - // get type of nestedList - Class classType = nestedList[0].getClass(); - - // fill outer list with nested list - int offset = 0; - int length; - if (fieldIdx == -1) { // set list of list as an object - for (int i = 0; offset < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - System.arraycopy(nestedList, offset, temp, 0, length); - offset = offset + length; - rows[i] = temp; - - } - } else { // set list of list as an field on Row - for (int i = 0; offset < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - System.arraycopy(nestedList, offset, temp, 0, length); - offset = offset + length; - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - - } - - private static void readNonNullMapColumn(Object[] rows, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { - - List fieldType = schema.getChildren(); - TypeDescription keyType = fieldType.get(0); - TypeDescription valueType = fieldType.get(1); - - ColumnVector keys = map.keys; - ColumnVector values = map.values; - Object[] keyRows = new Object[map.childCount]; - Object[] valueRows = new Object[map.childCount]; - - // read map kes and values - readField(keyRows, -1, keyType, keys, null, keyRows.length); - readField(valueRows, -1, valueType, values, null, valueRows.length); - - // check if map is directly in a list or not, e.g, array> - if (lengthVector == null) { - long[] lengthVectorMap = map.lengths; - int offset = 0; - if (fieldIdx == -1) { - for (int i = 0; i < childCount; i++) { - rows[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); - offset += lengthVectorMap[i]; - } - } else { - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); - offset += lengthVectorMap[i]; - } - } - } else { // list of map - - long[] lengthVectorMap = map.lengths; - int mapOffset = 0; // offset of map element - int offset = 0; // offset of map - HashMap[] temp; - if (fieldIdx == -1) { // set map list as an object - for (int i = 0; offset < childCount; i++) { - temp = new HashMap[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); - mapOffset += lengthVectorMap[offset]; - offset++; - } - rows[i] = temp; - } - } else { // set map list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new HashMap[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); - mapOffset += lengthVectorMap[offset]; - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - - private static void fillColumnWithRepeatingValue(Object[] rows, int fieldIdx, Object repeatingValue, int childCount) { - - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - rows[i] = repeatingValue; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - ((Row) rows[i]).setField(fieldIdx, repeatingValue); - } - } - } - - private static void fillListWithRepeatingNull(Object[] rows, int fieldIdx, Class classType, long[] lengthVector, int childCount) { - - int length; - if (fieldIdx == -1) { - for (int i = 0; i < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - rows[i] = temp; - } - } else { - for (int i = 0; i < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - - private static void readBooleanColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = vector.vector[i] != 0; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, vector.vector[i] != 0); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, boolean[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - boolean[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++] != 0; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new boolean[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++] != 0; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readByteColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = (byte) vector.vector[i]; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, (byte) vector.vector[i]); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, byte[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - byte[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (byte) vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (byte) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readShortColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = (short) vector.vector[i]; - } - } else { // set as field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, (short) vector.vector[i]); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, short[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - short[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (short) vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new short[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (short) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readIntColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = (int) vector.vector[i]; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, (int) vector.vector[i]); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, int[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - int[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (int) vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new int[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (int) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readLongColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = vector.vector[i]; - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); - } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, long[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - long[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new long[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readFloatColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { - boolean[] isNullVector = vector.isNull; + boolean[] isNullVector = bytes.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); } - rows[i] = (float) vector.vector[i]; } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); } - ((Row) rows[i]).setField(fieldIdx, (float) vector.vector[i]); } } } } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, float[].class, lengthVector, childCount); + if (bytes.isRepeating) { // fill list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::stringArray); } else { - // column contain null values int offset = 0; - float[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (float) vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; i < childCount; i++) { - temp = new float[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = (float) vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readDoubleColumn(Object[] rows, int fieldIdx, DoubleColumnVector vector, long[] lengthVector, int childCount) { - - // check if data type(dt) is directly in list or not, e.g, array
- if (lengthVector == null) { - if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - rows[i] = null; - continue; - } - rows[i] = vector.vector[i]; - } - } else { // set as field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + String[] temp; + boolean[] isNullVector = bytes.isNull; + for (int i = 0; offset < childCount; i++) { + temp = new String[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + } else { + temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + offset++; } - ((Row) rows[i]).setField(fieldIdx, vector.vector[i]); } - } - } - } else { // in a list - if (vector.isRepeating) { // // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, double[].class, lengthVector, childCount); - } else { - // column contain null values - int offset = 0; - double[] temp; - boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++]; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new double[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - continue; - } - temp[j] = vector.vector[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readStringColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { + private static void readBytesColumnAsBinary(Object[] vals, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { - // check if string is directly in a list or not, e.g, array + // check if the binary need to be read into lists or as single values if (lengthVector == null) { if (bytes.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { boolean[] isNullVector = bytes.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); } - rows[i] = new String(bytes.vector[i], bytes.start[i], bytes.length[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); } - ((Row) rows[i]).setField(fieldIdx, new String(bytes.vector[i], bytes.start[i], bytes.length[i])); } } } - } else { // in a list - if (bytes.isRepeating) { // fill list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, String[].class, lengthVector, childCount); + } else { + if (bytes.isRepeating) { // fill complete list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::binaryArray); } else { int offset = 0; - String[] temp; + byte[][] temp; boolean[] isNullVector = bytes.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; // skip null value - } - temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + for (int i = 0; offset < childCount; i++) { + temp = new byte[(int) lengthVector[i]][]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { offset++; - } - rows[i] = temp; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new String[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; // skip null value - } - temp[j] = new String(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); + } else { + temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); offset++; } - ((Row) rows[i]).setField(fieldIdx, temp); + } + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - } - private static void readDateColumn(Object[] rows, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { + private static void readLongColumnAsDate(Object[] vals, int fieldIdx, LongColumnVector vector, long[] lengthVector, int childCount) { - // check if date is directly in a list or not, e.g, array + // check if the values need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = readDate(vector.vector[i]); } - rows[i] = readDate(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, readDate(vector.vector[i])); } - ((Row) rows[i]).setField(fieldIdx, readDate(vector.vector[i])); } } } - } else { - if (vector.isRepeating) { // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, Date[].class, lengthVector, childCount); + } else { // in a list + if (vector.isRepeating) { // // fill complete list with first value + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::dateArray); } else { + // column contain null values int offset = 0; Date[] temp; boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = readDate(vector.vector[offset++]); - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Date[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } + for (int i = 0; offset < childCount; i++) { + temp = new Date[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + } else { temp[j] = readDate(vector.vector[offset++]); } - ((Row) rows[i]).setField(fieldIdx, temp); + } + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - } - private static void readTimestampColumn(Object[] rows, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { + private static void readTimestampColumn(Object[] vals, int fieldIdx, TimestampColumnVector vector, long[] lengthVector, int childCount) { - // check if timestamp is directly in a list or not, e.g, array + // check if the timestamps need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + Timestamp ts = readTimestamp(vector.time[i], vector.nanos[i]); + vals[i] = ts; } - Timestamp ts = new Timestamp(vector.time[i]); - ts.setNanos(vector.nanos[i]); - rows[i] = ts; } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + Timestamp ts = readTimestamp(vector.time[i], vector.nanos[i]); + rows[i].setField(fieldIdx, ts); } - Timestamp ts = new Timestamp(vector.time[i]); - ts.setNanos(vector.nanos[i]); - ((Row) rows[i]).setField(fieldIdx, ts); } } } - } - else { + } else { if (vector.isRepeating) { // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, Timestamp[].class, lengthVector, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::timestampArray); } else { int offset = 0; Timestamp[] temp; boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = new Timestamp(vector.time[offset]); - temp[j].setNanos(vector.nanos[offset]); + for (int i = 0; offset < childCount; i++) { + temp = new Timestamp[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { offset++; - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Timestamp[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = new Timestamp(vector.time[offset]); - temp[j].setNanos(vector.nanos[offset]); + } else { + temp[j] = readTimestamp(vector.time[offset], vector.nanos[offset]); offset++; } - ((Row) rows[i]).setField(fieldIdx, temp); - } - } - } - } - } - - private static void readBinaryColumn(Object[] rows, int fieldIdx, BytesColumnVector bytes, long[] lengthVector, int childCount) { - - // check if string is directly in a list or not, e.g, array - if (lengthVector == null) { - if (bytes.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); - } else { - boolean[] isNullVectorIndex = bytes.isNull; - if (fieldIdx == -1) { // set as an object - for (int i = 0; i < childCount; i++) { - if (isNullVectorIndex[i]) { - rows[i] = null; - continue; - } - rows[i] = readBinary(bytes.vector[i], bytes.start[i], bytes.length[i]); - } - } else { // set as a field of Row - for (int i = 0; i < childCount; i++) { - if (isNullVectorIndex[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; - } - ((Row) rows[i]).setField(fieldIdx, readBinary(bytes.vector[i], bytes.start[i], bytes.length[i])); } - } - } - } else { - if (bytes.isRepeating) { // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, byte[][].class, lengthVector, childCount); - } else { - int offset = 0; - byte[][] temp; - boolean[] isNullVector = bytes.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - rows[i] = temp; - } - } else { // set list as a field - for (int i = 0; offset < childCount; i++) { - temp = new byte[(int) lengthVector[i]][]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = readBinary(bytes.vector[offset], bytes.start[offset], bytes.length[offset]); - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readDecimalColumn(Object[] rows, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { + private static void readDecimalColumn(Object[] vals, int fieldIdx, DecimalColumnVector vector, long[] lengthVector, int childCount) { - // check if decimal is directly in a list or not, e.g, array + // check if the decimals need to be read into lists or as single values if (lengthVector == null) { if (vector.isRepeating) { // fill complete column with first value - // Also column contains null value and it's repeating - fillColumnWithRepeatingValue(rows, fieldIdx, null, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillColumnWithRepeatingValue(vals, fieldIdx, null, childCount); } else { boolean[] isNullVector = vector.isNull; if (fieldIdx == -1) { // set as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = readBigDecimal(vector.vector[i]); } - rows[i] = readBigDecimal(vector.vector[i]); } } else { // set as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, readBigDecimal(vector.vector[i])); } - ((Row) rows[i]).setField(fieldIdx, readBigDecimal(vector.vector[i])); } } } } else { if (vector.isRepeating) { // fill complete list with first value - // Also column contains null value and it's repeating - // so all values are null, but we need to set list with null values - fillListWithRepeatingNull(rows, fieldIdx, BigDecimal[].class, lengthVector, childCount); + // since the column contains null values and has just one distinct value, the repeated value is null + fillListWithRepeatingNull(vals, fieldIdx, lengthVector, childCount, OrcUtils::decimalArray); } else { int offset = 0; BigDecimal[] temp; boolean[] isNullVector = vector.isNull; - if (fieldIdx == -1) { // set list as an object - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } - temp[j] = readBigDecimal(vector.vector[offset++]); - } - rows[i] = temp; - } - } else { // set list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new BigDecimal[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - offset++; - temp[j] = null; - continue; - } + for (int i = 0; offset < childCount; i++) { + temp = new BigDecimal[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + offset++; + } else { temp[j] = readBigDecimal(vector.vector[offset++]); } - ((Row) rows[i]).setField(fieldIdx, temp); + } + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } } - private static void readStructColumn(Object[] rows, int fieldIdx, StructColumnVector struct, TypeDescription schema, long[] lengthVector, int childCount) { + private static void readStructColumn(Object[] vals, int fieldIdx, StructColumnVector structVector, TypeDescription schema, long[] lengthVector, int childCount) { List childrenTypes = schema.getChildren(); - int numChildren = childrenTypes.size(); - Row[] nestedFields = new Row[childCount]; + int numFields = childrenTypes.size(); + // create a batch of Rows to read the structs + Row[] structs = new Row[childCount]; + // TODO: possible improvement: reuse existing Row objects for (int i = 0; i < childCount; i++) { - nestedFields[i] = new Row(numChildren); + structs[i] = new Row(numFields); } - for (int i = 0; i < numChildren; i++) { - readField(nestedFields, i, childrenTypes.get(i), struct.fields[i], null, childCount); + + // read struct fields + for (int i = 0; i < numFields; i++) { + readField(structs, i, childrenTypes.get(i), structVector.fields[i], null, childCount); } - boolean[] isNullVector = struct.isNull; + boolean[] isNullVector = structVector.isNull; - // check if struct is directly in a list or not, e.g, array> + // check if the structs need to be read into lists or as single values if (lengthVector == null) { if (fieldIdx == -1) { // set struct as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = structs[i]; } - rows[i] = nestedFields[i]; } } else { // set struct as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, structs[i]); } - ((Row) rows[i]).setField(fieldIdx, nestedFields[i]); } } } else { // struct in a list int offset = 0; Row[] temp; - if (fieldIdx == -1) { // set list of struct as an object - for (int i = 0; offset < childCount; i++) { - temp = new Row[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - temp[j] = null; - continue; - } - temp[j] = nestedFields[offset++]; + for (int i = 0; offset < childCount; i++) { + temp = new Row[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + } else { + temp[j] = structs[offset++]; } - rows[i] = temp; } - } - else { // set list of struct as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new Row[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - temp[j] = null; - continue; - } - temp[j] = nestedFields[offset++]; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { // set list of structs as an object + vals[i] = temp; + } else { // set list of structs as field of row + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - private static void readListColumn(Object[] rows, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { + private static void readListColumn(Object[] vals, int fieldIdx, ListColumnVector list, TypeDescription schema, long[] lengthVector, int childCount) { TypeDescription fieldType = schema.getChildren().get(0); + // check if the lists need to be read into lists or as single values if (lengthVector == null) { long[] lengthVectorNested = list.lengths; - readField(rows, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); - } - else { // list in a list - + readField(vals, fieldIdx, fieldType, list.child, lengthVectorNested, list.childCount); + } else { // list in a list Object[] nestedList = new Object[childCount]; - // length vector for nested list long[] lengthVectorNested = list.lengths; - // read nested list readField(nestedList, -1, fieldType, list.child, lengthVectorNested, list.childCount); - // get type of nestedList - Class classType = nestedList[0].getClass(); - // fill outer list with nested list int offset = 0; int length; - if (fieldIdx == -1) { // set list of list as an object - for (int i = 0; offset < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - System.arraycopy(nestedList, offset, temp, 0, length); - offset = offset + length; - rows[i] = temp; - - } - } else { // set list of list as an field on Row - for (int i = 0; offset < childCount; i++) { - length = (int) lengthVector[i]; - Object temp = Array.newInstance(classType, length); - System.arraycopy(nestedList, offset, temp, 0, length); - offset = offset + length; - ((Row) rows[i]).setField(fieldIdx, temp); + // get type of nestedList + Class classType = nestedList[0].getClass(); + for (int i = 0; offset < childCount; i++) { + length = (int) lengthVector[i]; + Object[] temp = (Object[]) Array.newInstance(classType, length); + System.arraycopy(nestedList, offset, temp, 0, length); + offset = offset + length; + if (fieldIdx == -1) { // set list of list as an object + vals[i] = temp; + } else { // set list of list as field of row + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - private static void readMapColumn(Object[] rows, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { + private static void readMapColumn(Object[] vals, int fieldIdx, MapColumnVector map, TypeDescription schema, long[] lengthVector, int childCount) { List fieldType = schema.getChildren(); TypeDescription keyType = fieldType.get(0); @@ -2134,27 +1296,28 @@ private static void readMapColumn(Object[] rows, int fieldIdx, MapColumnVector m boolean[] isNullVector = map.isNull; - // check if map is directly in a list or not, e.g, array> + // check if the maps need to be read into lists or as single values if (lengthVector == null) { long[] lengthVectorMap = map.lengths; int offset = 0; if (fieldIdx == -1) { // set map as an object for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - rows[i] = null; - continue; + vals[i] = null; + } else { + vals[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); + offset += lengthVectorMap[i]; } - rows[i] = readHashMap(keyRows, valueRows, offset, lengthVectorMap[i]); - offset += lengthVectorMap[i]; } } else { // set map as a field of Row + Row[] rows = (Row[]) vals; for (int i = 0; i < childCount; i++) { if (isNullVector[i]) { - ((Row) rows[i]).setField(fieldIdx, null); - continue; + rows[i].setField(fieldIdx, null); + } else { + rows[i].setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); + offset += lengthVectorMap[i]; } - ((Row) rows[i]).setField(fieldIdx, readHashMap(keyRows, valueRows, offset, lengthVectorMap[i])); - offset += lengthVectorMap[i]; } } } else { // list of map @@ -2162,41 +1325,110 @@ private static void readMapColumn(Object[] rows, int fieldIdx, MapColumnVector m int mapOffset = 0; // offset of map element int offset = 0; // offset of map HashMap[] temp; - if (fieldIdx == -1) { // set map list as an object - for (int i = 0; offset < childCount; i++) { - temp = new HashMap[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - temp[j] = null; - continue; - } + + for (int i = 0; offset < childCount; i++) { + temp = new HashMap[(int) lengthVector[i]]; + for (int j = 0; j < temp.length; j++) { + if (isNullVector[offset]) { + temp[j] = null; + } else { temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); mapOffset += lengthVectorMap[offset]; offset++; } - rows[i] = temp; } - } else { // set map list as a field of Row - for (int i = 0; offset < childCount; i++) { - temp = new HashMap[(int) lengthVector[i]]; - for (int j = 0; j < temp.length; j++) { - if (isNullVector[offset]) { - temp[j] = null; - continue; - } - temp[j] = readHashMap(keyRows, valueRows, mapOffset, lengthVectorMap[offset]); - mapOffset += lengthVectorMap[offset]; - offset++; - } - ((Row) rows[i]).setField(fieldIdx, temp); + if (fieldIdx == -1) { + vals[i] = temp; + } else { + ((Row) vals[i]).setField(fieldIdx, temp); } } } } - private static BigDecimal readBigDecimal(HiveDecimalWritable hiveDecimalWritable) { - HiveDecimal hiveDecimal = hiveDecimalWritable.getHiveDecimal(); - return hiveDecimal.bigDecimalValue(); + /** + * Sets a repeating value to all objects or row fields of the passed vals array. + * + * @param vals The array of objects or Rows. + * @param fieldIdx If the objs array is an array of Row, the index of the field that needs to be filled. + * Otherwise a -1 must be passed and the data is directly filled into the array. + * @param repeatingValue The value that is set. + * @param childCount The number of times the value is set. + */ + private static void fillColumnWithRepeatingValue(Object[] vals, int fieldIdx, Object repeatingValue, int childCount) { + + if (fieldIdx == -1) { + // set value as an object + Arrays.fill(vals, 0, childCount, repeatingValue); + } else { + // set value as a field of Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + rows[i].setField(fieldIdx, repeatingValue); + } + } + } + + /** + * Sets arrays containing only null values to all objects or row fields of the passed vals array. + * + * @param vals The array of objects or Rows to which the empty arrays are set. + * @param fieldIdx If the objs array is an array of Row, the index of the field that needs to be filled. + * Otherwise a -1 must be passed and the data is directly filled into the array. + * @param lengthVector The vector containing the lengths of the individual empty arrays. + * @param childCount The number of objects or Rows to fill. + * @param array A method to create arrays of the appropriate type. + * @param The type of the arrays to create. + */ + private static void fillListWithRepeatingNull(Object[] vals, int fieldIdx, long[] lengthVector, int childCount, IntFunction array) { + + if (fieldIdx == -1) { + // set empty array as object + for (int i = 0; i < childCount; i++) { + vals[i] = array.apply((int) lengthVector[i]); + } + } else { + // set empty array as field in Row + Row[] rows = (Row[]) vals; + for (int i = 0; i < childCount; i++) { + rows[i].setField(fieldIdx, array.apply((int) lengthVector[i])); + } + } + } + + private static Boolean readBoolean(long l) { + return l != 0; + } + + private static Byte readByte(long l) { + return (byte) l; + } + + private static Short readShort(long l) { + return (short) l; + } + + private static Integer readInt(long l) { + return (int) l; + } + + private static Long readLong(long l) { + return l; + } + + private static Float readFloat(double d) { + return (float) d; + } + + private static Double readDouble(double d) { + return d; + } + + private static Date readDate(long l) { + // day to milliseconds + final long t = l * MILLIS_PER_DAY; + // adjust by local timezone + return new java.sql.Date(t - LOCAL_TZ.getOffset(t)); } private static byte[] readBinary(byte[] src, int srcPos, int length) { @@ -2205,19 +1437,18 @@ private static byte[] readBinary(byte[] src, int srcPos, int length) { return result; } - private static Timestamp readTimeStamp(long time, int nanos) { + private static BigDecimal readBigDecimal(HiveDecimalWritable hiveDecimalWritable) { + HiveDecimal hiveDecimal = hiveDecimalWritable.getHiveDecimal(); + return hiveDecimal.bigDecimalValue(); + } + + private static Timestamp readTimestamp(long time, int nanos) { Timestamp ts = new Timestamp(time); ts.setNanos(nanos); return ts; } - private static Date readDate(long days) { - // day to milliseconds - return new Date(days * 24 * 60 * 60 * 1000); - } - private static HashMap readHashMap(Object[] keyRows, Object[] valueRows, int offset, long length) { - HashMap resultMap = new HashMap<>(); for (int j = 0; j < length; j++) { resultMap.put(keyRows[offset], valueRows[offset]); @@ -2226,4 +1457,52 @@ private static HashMap readHashMap(Object[] keyRows, Object[] valueRows, int off return resultMap; } + private static Boolean[] boolArray(int len) { + return new Boolean[len]; + } + + private static Byte[] byteArray(int len) { + return new Byte[len]; + } + + private static Short[] shortArray(int len) { + return new Short[len]; + } + + private static Integer[] intArray(int len) { + return new Integer[len]; + } + + private static Long[] longArray(int len) { + return new Long[len]; + } + + private static Float[] floatArray(int len) { + return new Float[len]; + } + + private static Double[] doubleArray(int len) { + return new Double[len]; + } + + private static Date[] dateArray(int len) { + return new Date[len]; + } + + private static byte[][] binaryArray(int len) { + return new byte[len][]; + } + + private static String[] stringArray(int len) { + return new String[len]; + } + + private static BigDecimal[] decimalArray(int len) { + return new BigDecimal[len]; + } + + private static Timestamp[] timestampArray(int len) { + return new Timestamp[len]; + } + } diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java deleted file mode 100644 index 0c9c549aed2cf..0000000000000 --- a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/RowOrcInputFormat.java +++ /dev/null @@ -1,241 +0,0 @@ -/* - * 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.flink.orc; - -import org.apache.flink.api.common.io.FileInputFormat; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; - -import org.apache.orc.OrcConf; -import org.apache.orc.OrcFile; -import org.apache.orc.Reader; -import org.apache.orc.RecordReader; -import org.apache.orc.TypeDescription; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; - -import static org.apache.flink.orc.OrcUtils.fillRows; - -/** - * InputFormat to read ORC data. - * For Optimization, reading is done in batch instead of a single row. - */ -public class RowOrcInputFormat - extends FileInputFormat - implements ResultTypeQueryable { - - private static final Logger LOG = LoggerFactory.getLogger(RowOrcInputFormat.class); - private static final int BATCH_SIZE = 1024; - - private org.apache.hadoop.conf.Configuration config; - private TypeDescription schema; - private int[] fieldMapping; - - private transient RowTypeInfo rowType; - private transient RecordReader orcRowsReader; - private transient VectorizedRowBatch rowBatch; - private transient Row[] rows; - - private transient int rowInBatch; - - public RowOrcInputFormat(String path, String schemaString, Configuration orcConfig) { - this(path, TypeDescription.fromString(schemaString), orcConfig); - } - - public RowOrcInputFormat(String path, TypeDescription orcSchema, Configuration orcConfig) { - super(new Path(path)); - this.unsplittable = false; - this.schema = orcSchema; - this.rowType = (RowTypeInfo) OrcUtils.schemaToTypeInfo(schema); - this.config = orcConfig; - - this.fieldMapping = new int[this.schema.getChildren().size()]; - for (int i = 0; i < fieldMapping.length; i++) { - this.fieldMapping[i] = i; - } - - } - - public void setFieldMapping(int[] fieldMapping) { - this.fieldMapping = fieldMapping; - // adapt result type - - TypeInformation[] fieldTypes = new TypeInformation[fieldMapping.length]; - String[] fieldNames = new String[fieldMapping.length]; - for (int i = 0; i < fieldMapping.length; i++) { - fieldTypes[i] = this.rowType.getTypeAt(fieldMapping[i]); - fieldNames[i] = this.rowType.getFieldNames()[fieldMapping[i]]; - } - this.rowType = new RowTypeInfo(fieldTypes, fieldNames); - } - - private boolean[] computeProjectionMask() { - boolean[] projectionMask = new boolean[schema.getMaximumId() + 1]; - for (int inIdx : fieldMapping) { - TypeDescription fieldSchema = schema.getChildren().get(inIdx); - for (int i = fieldSchema.getId(); i <= fieldSchema.getMaximumId(); i++) { - projectionMask[i] = true; - } - } - return projectionMask; - } - - @Override - public void openInputFormat() throws IOException { - super.openInputFormat(); - this.rows = new Row[BATCH_SIZE]; - for (int i = 0; i < BATCH_SIZE; i++) { - rows[i] = new Row(fieldMapping.length); - } - } - - @Override - public void open(FileInputSplit fileSplit) throws IOException { - - this.currentSplit = fileSplit; - Preconditions.checkArgument(this.splitStart == 0, "ORC files must be read from the start."); - - if (LOG.isDebugEnabled()) { - LOG.debug("Opening ORC file " + fileSplit.getPath()); - } - - org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(fileSplit.getPath().getPath()); - - Reader orcReader = OrcFile.createReader(hPath, OrcFile.readerOptions(config)); - - Reader.Options options = orcReader.options() - .range(fileSplit.getStart(), fileSplit.getLength()) - .useZeroCopy(OrcConf.USE_ZEROCOPY.getBoolean(config)) - .skipCorruptRecords(OrcConf.SKIP_CORRUPT_DATA.getBoolean(config)) - .tolerateMissingSchema(OrcConf.TOLERATE_MISSING_SCHEMA.getBoolean(config)); - - options.include(computeProjectionMask()); - - // check that schema of file is as expected - if (!this.schema.equals(orcReader.getSchema())) { - - throw new RuntimeException("Invalid schema for file at " + this.filePath + - " Expected:" + this.schema + " Actual: " + orcReader.getSchema()); - } - - this.orcRowsReader = orcReader.rows(options); - - // assign ids - this.schema.getId(); - - this.rowBatch = schema.createRowBatch(BATCH_SIZE); - rowInBatch = 0; - } - - @Override - public void close() throws IOException { - - if (orcRowsReader != null) { - this.orcRowsReader.close(); - } - this.orcRowsReader = null; - - } - - @Override - public void closeInputFormat() throws IOException { - this.rows = null; - this.rows = null; - this.schema = null; - this.rowBatch = null; - } - - @Override - public boolean reachedEnd() throws IOException { - return !ensureBatch(); - } - - private boolean ensureBatch() throws IOException { - - if (rowInBatch >= rowBatch.size) { - rowInBatch = 0; - boolean moreRows = orcRowsReader.nextBatch(rowBatch); - - if (moreRows) { - // read rows - fillRows(rows, schema, rowBatch, fieldMapping); - } - return moreRows; - } - - return true; - } - - @Override - public Row nextRecord(Row reuse) throws IOException { - return rows[this.rowInBatch++]; - } - - @Override - public TypeInformation getProducedType() { - return rowType; - } - - // -------------------------------------------------------------------------------------------- - // Custom serialization methods - // -------------------------------------------------------------------------------------------- - - private void writeObject(ObjectOutputStream out) throws IOException { - this.config.write(out); - out.writeUTF(schema.toString()); - - out.writeInt(fieldMapping.length); - for (int f : fieldMapping) { - out.writeInt(f); - } - - } - - @SuppressWarnings("unchecked") - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - - org.apache.hadoop.conf.Configuration configuration = new org.apache.hadoop.conf.Configuration(); - configuration.readFields(in); - - if (this.config == null) { - this.config = configuration; - } - this.schema = TypeDescription.fromString(in.readUTF()); - - this.fieldMapping = new int[in.readInt()]; - for (int i = 0; i < fieldMapping.length; i++) { - this.fieldMapping[i] = in.readInt(); - } - - } - -} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java new file mode 100644 index 0000000000000..0efe41fd1525d --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java @@ -0,0 +1,795 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.orc.Reader; +import org.apache.orc.StripeInformation; +import org.junit.After; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link OrcRowInputFormat}. + * + */ +public class OrcRowInputFormatTest { + + private OrcRowInputFormat rowOrcInputFormat; + + @After + public void tearDown() throws IOException { + if (rowOrcInputFormat != null) { + rowOrcInputFormat.close(); + rowOrcInputFormat.closeInputFormat(); + } + rowOrcInputFormat = null; + } + + private static final String TEST_FILE_FLAT = "test-data-flat.orc"; + private static final String TEST_SCHEMA_FLAT = + "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int,_col5:string,_col6:int,_col7:int,_col8:int>"; + + private static final String TEST_FILE_NESTED = "test-data-nested.orc"; + private static final String TEST_SCHEMA_NESTED = + "struct<" + + "boolean1:boolean," + + "byte1:tinyint," + + "short1:smallint," + + "int1:int," + + "long1:bigint," + + "float1:float," + + "double1:double," + + "bytes1:binary," + + "string1:string," + + "middle:struct<" + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">," + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">," + + "map:map<" + + "string," + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">"; + + private static final String TEST_FILE_TIMETYPES = "test-data-timetypes.orc"; + private static final String TEST_SCHEMA_TIMETYPES = "struct"; + + private static final String TEST_FILE_DECIMAL = "test-data-decimal.orc"; + private static final String TEST_SCHEMA_DECIMAL = "struct<_col0:decimal(10,5)>"; + + private static final String TEST_FILE_NESTEDLIST = "test-data-nestedlist.orc"; + private static final String TEST_SCHEMA_NESTEDLIST = "struct>>>"; + + @Test(expected = FileNotFoundException.class) + public void testInvalidPath() throws IOException{ + rowOrcInputFormat = + new OrcRowInputFormat("/does/not/exist", TEST_SCHEMA_FLAT, new Configuration()); + rowOrcInputFormat.openInputFormat(); + FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.open(inputSplits[0]); + } + + @Test(expected = IndexOutOfBoundsException.class) + public void testInvalidProjection1() throws IOException{ + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + int[] projectionMask = {1, 2, 3, -1}; + rowOrcInputFormat.selectFields(projectionMask); + } + + @Test(expected = IndexOutOfBoundsException.class) + public void testInvalidProjection2() throws IOException{ + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + int[] projectionMask = {1, 2, 3, 9}; + rowOrcInputFormat.selectFields(projectionMask); + } + + @Test + public void testProjectionMaskNested() throws IOException{ + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + OrcRowInputFormat spy = spy(rowOrcInputFormat); + + // mock options to check configuration of ORC reader + Reader.Options options = new Reader.Options(); + doReturn(options).when(spy).getOptions(any()); + + spy.selectFields(9, 11, 2); + spy.openInputFormat(); + FileInputSplit[] splits = spy.createInputSplits(1); + spy.open(splits[0]); + + // top-level struct is false + boolean[] expected = new boolean[]{ + false, // top level + false, false, // flat fields 0, 1 are out + true, // flat field 2 is in + false, false, false, false, false, false, // flat fields 3, 4, 5, 6, 7, 8 are out + true, true, true, true, true, // nested field 9 is in + false, false, false, false, // nested field 10 is out + true, true, true, true, true}; // nested field 11 is in + assertArrayEquals(expected, options.getInclude()); + } + + @Test + public void testSplitStripesGivenSplits() throws IOException { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + + OrcRowInputFormat spy = spy(rowOrcInputFormat); + + // mock options to check configuration of ORC reader + Reader.Options options = spy(new Reader.Options()); + doReturn(options).when(spy).getOptions(any()); + + FileInputSplit[] splits = spy.createInputSplits(3); + + spy.openInputFormat(); + spy.open(splits[0]); + verify(options).range(eq(3L), eq(137005L)); + spy.open(splits[1]); + verify(options).range(eq(137008L), eq(136182L)); + spy.open(splits[2]); + verify(options).range(eq(273190L), eq(123633L)); + } + + @Test + public void testSplitStripesCustomSplits() throws IOException { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + + OrcRowInputFormat spy = spy(rowOrcInputFormat); + + // mock list of stripes + List stripes = new ArrayList<>(); + StripeInformation stripe1 = mock(StripeInformation.class); + when(stripe1.getOffset()).thenReturn(10L); + when(stripe1.getLength()).thenReturn(90L); + StripeInformation stripe2 = mock(StripeInformation.class); + when(stripe2.getOffset()).thenReturn(100L); + when(stripe2.getLength()).thenReturn(100L); + StripeInformation stripe3 = mock(StripeInformation.class); + when(stripe3.getOffset()).thenReturn(200L); + when(stripe3.getLength()).thenReturn(100L); + StripeInformation stripe4 = mock(StripeInformation.class); + when(stripe4.getOffset()).thenReturn(300L); + when(stripe4.getLength()).thenReturn(100L); + StripeInformation stripe5 = mock(StripeInformation.class); + when(stripe5.getOffset()).thenReturn(400L); + when(stripe5.getLength()).thenReturn(100L); + stripes.add(stripe1); + stripes.add(stripe2); + stripes.add(stripe3); + stripes.add(stripe4); + stripes.add(stripe5); + doReturn(stripes).when(spy).getStripes(any()); + + // mock options to check configuration of ORC reader + Reader.Options options = spy(new Reader.Options()); + doReturn(options).when(spy).getOptions(any()); + + spy.openInputFormat(); + // split ranging 2 stripes + spy.open(new FileInputSplit(0, new Path(getPath(TEST_FILE_FLAT)), 0, 150, new String[]{})); + verify(options).range(eq(10L), eq(190L)); + // split ranging 0 stripes + spy.open(new FileInputSplit(1, new Path(getPath(TEST_FILE_FLAT)), 150, 10, new String[]{})); + verify(options).range(eq(0L), eq(0L)); + // split ranging 1 stripe + spy.open(new FileInputSplit(2, new Path(getPath(TEST_FILE_FLAT)), 160, 41, new String[]{})); + verify(options).range(eq(200L), eq(100L)); + // split ranging 2 stripe + spy.open(new FileInputSplit(3, new Path(getPath(TEST_FILE_FLAT)), 201, 299, new String[]{})); + verify(options).range(eq(300L), eq(200L)); + } + + @Test + public void testProducedType() throws IOException { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + assertTrue(rowOrcInputFormat.getProducedType() instanceof RowTypeInfo); + RowTypeInfo producedType = (RowTypeInfo) rowOrcInputFormat.getProducedType(); + + assertArrayEquals( + new TypeInformation[]{ + // primitives + Types.BOOLEAN, Types.BYTE, Types.SHORT, Types.INT, Types.LONG, Types.FLOAT, Types.DOUBLE, + // binary + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + // string + Types.STRING, + // struct + Types.ROW_NAMED( + new String[]{"list"}, + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING))), + // list + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING)), + // map + new MapTypeInfo<>(Types.STRING, Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING)) + }, + producedType.getFieldTypes()); + assertArrayEquals( + new String[]{"boolean1", "byte1", "short1", "int1", "long1", "float1", "double1", "bytes1", "string1", "middle", "list", "map"}, + producedType.getFieldNames()); + } + + @Test + public void testProducedTypeWithProjection() throws IOException { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + rowOrcInputFormat.selectFields(9, 3, 7, 10); + + assertTrue(rowOrcInputFormat.getProducedType() instanceof RowTypeInfo); + RowTypeInfo producedType = (RowTypeInfo) rowOrcInputFormat.getProducedType(); + + assertArrayEquals( + new TypeInformation[]{ + // struct + Types.ROW_NAMED( + new String[]{"list"}, + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING))), + // int + Types.INT, + // binary + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + // list + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[]{"int1", "string1"}, Types.INT, Types.STRING)) + }, + producedType.getFieldTypes()); + assertArrayEquals( + new String[]{"middle", "int1", "bytes1", "list"}, + producedType.getFieldNames()); + } + + @Test + public void testSerialization() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + + rowOrcInputFormat.selectFields(0, 4, 1); + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("_col1", PredicateLeaf.Type.STRING, "M")); + + byte[] bytes = InstantiationUtil.serializeObject(rowOrcInputFormat); + OrcRowInputFormat copy = InstantiationUtil.deserializeObject(bytes, getClass().getClassLoader()); + + FileInputSplit[] splits = copy.createInputSplits(1); + copy.openInputFormat(); + copy.open(splits[0]); + assertFalse(copy.reachedEnd()); + Row row = copy.nextRecord(null); + + assertNotNull(row); + assertEquals(3, row.getArity()); + // check first row + assertEquals(1, row.getField(0)); + assertEquals(500, row.getField(1)); + assertEquals("M", row.getField(2)); + } + + @Test + public void testNumericBooleanStringPredicates() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + rowOrcInputFormat.selectFields(0, 1, 2, 3, 4, 5, 6, 8); + + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("boolean1", PredicateLeaf.Type.BOOLEAN, false)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.LessThan("byte1", PredicateLeaf.Type.LONG, 1)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.LessThanEquals("short1", PredicateLeaf.Type.LONG, 1024)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Between("int1", PredicateLeaf.Type.LONG, -1, 65536)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("long1", PredicateLeaf.Type.LONG, 9223372036854775807L)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("float1", PredicateLeaf.Type.FLOAT, 1.0)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("double1", PredicateLeaf.Type.FLOAT, -15.0)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.IsNull("string1", PredicateLeaf.Type.STRING)); + // boolean pred + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("string1", PredicateLeaf.Type.STRING, "hello")); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.openInputFormat(); + + // mock options to check configuration of ORC reader + OrcRowInputFormat spy = spy(rowOrcInputFormat); + Reader.Options options = new Reader.Options(); + doReturn(options).when(spy).getOptions(any()); + + spy.openInputFormat(); + spy.open(splits[0]); + + // verify predicate configuration + SearchArgument sarg = options.getSearchArgument(); + assertNotNull(sarg); + assertEquals("(and leaf-0 leaf-1 leaf-2 leaf-3 leaf-4 leaf-5 leaf-6 leaf-7 leaf-8)", sarg.getExpression().toString()); + assertEquals(9, sarg.getLeaves().size()); + List leaves = sarg.getLeaves(); + assertEquals("(EQUALS boolean1 false)", leaves.get(0).toString()); + assertEquals("(LESS_THAN byte1 1)", leaves.get(1).toString()); + assertEquals("(LESS_THAN_EQUALS short1 1024)", leaves.get(2).toString()); + assertEquals("(BETWEEN int1 -1 65536)", leaves.get(3).toString()); + assertEquals("(EQUALS long1 9223372036854775807)", leaves.get(4).toString()); + assertEquals("(EQUALS float1 1.0)", leaves.get(5).toString()); + assertEquals("(EQUALS double1 -15.0)", leaves.get(6).toString()); + assertEquals("(IS_NULL string1)", leaves.get(7).toString()); + assertEquals("(EQUALS string1 hello)", leaves.get(8).toString()); + } + + @Test + public void testTimePredicates() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_TIMETYPES), TEST_SCHEMA_TIMETYPES, new Configuration()); + + rowOrcInputFormat.addPredicate( + // OR + new OrcRowInputFormat.Or( + // timestamp pred + new OrcRowInputFormat.Equals("time", PredicateLeaf.Type.TIMESTAMP, Timestamp.valueOf("1900-05-05 12:34:56.100")), + // date pred + new OrcRowInputFormat.Equals("date", PredicateLeaf.Type.DATE, Date.valueOf("1900-12-25"))) + ); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.openInputFormat(); + + // mock options to check configuration of ORC reader + OrcRowInputFormat spy = spy(rowOrcInputFormat); + Reader.Options options = new Reader.Options(); + doReturn(options).when(spy).getOptions(any()); + + spy.openInputFormat(); + spy.open(splits[0]); + + // verify predicate configuration + SearchArgument sarg = options.getSearchArgument(); + assertNotNull(sarg); + assertEquals("(or leaf-0 leaf-1)", sarg.getExpression().toString()); + assertEquals(2, sarg.getLeaves().size()); + List leaves = sarg.getLeaves(); + assertEquals("(EQUALS time 1900-05-05 12:34:56.1)", leaves.get(0).toString()); + assertEquals("(EQUALS date 1900-12-25)", leaves.get(1).toString()); + } + + @Test + public void testDecimalPredicate() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_DECIMAL), TEST_SCHEMA_DECIMAL, new Configuration()); + + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Not( + // decimal pred + new OrcRowInputFormat.Equals("_col0", PredicateLeaf.Type.DECIMAL, BigDecimal.valueOf(-1000.5)))); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + rowOrcInputFormat.openInputFormat(); + + // mock options to check configuration of ORC reader + OrcRowInputFormat spy = spy(rowOrcInputFormat); + Reader.Options options = new Reader.Options(); + doReturn(options).when(spy).getOptions(any()); + + spy.openInputFormat(); + spy.open(splits[0]); + + // verify predicate configuration + SearchArgument sarg = options.getSearchArgument(); + assertNotNull(sarg); + assertEquals("(not leaf-0)", sarg.getExpression().toString()); + assertEquals(1, sarg.getLeaves().size()); + List leaves = sarg.getLeaves(); + assertEquals("(EQUALS _col0 -1000.5)", leaves.get(0).toString()); + } + + @Test(expected = IllegalArgumentException.class) + public void testPredicateWithInvalidColumn() throws Exception { + rowOrcInputFormat = + new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("unknown", PredicateLeaf.Type.LONG, 42)); + } + + @Test + public void testReadNestedFile() throws IOException{ + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + + // validate first row + assertNotNull(row); + assertEquals(12, row.getArity()); + assertEquals(false, row.getField(0)); + assertEquals((byte) 1, row.getField(1)); + assertEquals((short) 1024, row.getField(2)); + assertEquals(65536, row.getField(3)); + assertEquals(9223372036854775807L, row.getField(4)); + assertEquals(1.0f, row.getField(5)); + assertEquals(-15.0d, row.getField(6)); + assertArrayEquals(new byte[]{0, 1, 2, 3, 4}, (byte[]) row.getField(7)); + assertEquals("hi", row.getField(8)); + // check nested field + assertTrue(row.getField(9) instanceof Row); + Row nested1 = (Row) row.getField(9); + assertEquals(1, nested1.getArity()); + assertTrue(nested1.getField(0) instanceof Object[]); + Object[] nestedList1 = (Object[]) nested1.getField(0); + assertEquals(2, nestedList1.length); + assertEquals(Row.of(1, "bye"), nestedList1[0]); + assertEquals(Row.of(2, "sigh"), nestedList1[1]); + // check list + assertTrue(row.getField(10) instanceof Object[]); + Object[] list1 = (Object[]) row.getField(10); + assertEquals(2, list1.length); + assertEquals(Row.of(3, "good"), list1[0]); + assertEquals(Row.of(4, "bad"), list1[1]); + // check map + assertTrue(row.getField(11) instanceof HashMap); + HashMap map1 = (HashMap) row.getField(11); + assertEquals(0, map1.size()); + + // read second row + assertFalse(rowOrcInputFormat.reachedEnd()); + row = rowOrcInputFormat.nextRecord(null); + + // validate second row + assertNotNull(row); + assertEquals(12, row.getArity()); + assertEquals(true, row.getField(0)); + assertEquals((byte) 100, row.getField(1)); + assertEquals((short) 2048, row.getField(2)); + assertEquals(65536, row.getField(3)); + assertEquals(9223372036854775807L, row.getField(4)); + assertEquals(2.0f, row.getField(5)); + assertEquals(-5.0d, row.getField(6)); + assertArrayEquals(new byte[]{}, (byte[]) row.getField(7)); + assertEquals("bye", row.getField(8)); + // check nested field + assertTrue(row.getField(9) instanceof Row); + Row nested2 = (Row) row.getField(9); + assertEquals(1, nested2.getArity()); + assertTrue(nested2.getField(0) instanceof Object[]); + Object[] nestedList2 = (Object[]) nested2.getField(0); + assertEquals(2, nestedList2.length); + assertEquals(Row.of(1, "bye"), nestedList2[0]); + assertEquals(Row.of(2, "sigh"), nestedList2[1]); + // check list + assertTrue(row.getField(10) instanceof Object[]); + Object[] list2 = (Object[]) row.getField(10); + assertEquals(3, list2.length); + assertEquals(Row.of(100000000, "cat"), list2[0]); + assertEquals(Row.of(-100000, "in"), list2[1]); + assertEquals(Row.of(1234, "hat"), list2[2]); + // check map + assertTrue(row.getField(11) instanceof HashMap); + HashMap map = (HashMap) row.getField(11); + assertEquals(2, map.size()); + assertEquals(Row.of(5, "chani"), map.get("chani")); + assertEquals(Row.of(1, "mauddib"), map.get("mauddib")); + + assertTrue(rowOrcInputFormat.reachedEnd()); + } + + @Test + public void testReadTimeTypeFile() throws IOException{ + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_TIMETYPES), TEST_SCHEMA_TIMETYPES, new Configuration()); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + + // validate first row + assertNotNull(row); + assertEquals(2, row.getArity()); + assertEquals(Timestamp.valueOf("1900-05-05 12:34:56.1"), row.getField(0)); + assertEquals(Date.valueOf("1900-12-25"), row.getField(1)); + + // check correct number of rows + long cnt = 1; + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + assertEquals(70000, cnt); + } + + @Test + public void testReadDecimalTypeFile() throws IOException{ + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_DECIMAL), TEST_SCHEMA_DECIMAL, new Configuration()); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + + // validate first row + assertNotNull(row); + assertEquals(1, row.getArity()); + assertEquals(BigDecimal.valueOf(-1000.5d), row.getField(0)); + + // check correct number of rows + long cnt = 1; + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + assertEquals(6000, cnt); + } + + @Test + public void testReadNestedListFile() throws Exception { + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_NESTEDLIST), TEST_SCHEMA_NESTEDLIST, new Configuration()); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + + Row row = null; + long cnt = 0; + + // read all rows + while (!rowOrcInputFormat.reachedEnd()) { + + row = rowOrcInputFormat.nextRecord(row); + assertEquals(1, row.getArity()); + + // outer list + Object[] list = (Object[]) row.getField(0); + assertEquals(1, list.length); + + // nested list of rows + Row[] nestedRows = (Row[]) list[0]; + assertEquals(1, nestedRows.length); + assertEquals(1, nestedRows[0].getArity()); + + // verify list value + assertEquals(cnt, nestedRows[0].getField(0)); + cnt++; + } + // number of rows in file + assertEquals(100, cnt); + } + + @Test + public void testReadWithProjection() throws IOException{ + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_NESTED), TEST_SCHEMA_NESTED, new Configuration()); + + rowOrcInputFormat.selectFields(7, 0, 10, 8); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + rowOrcInputFormat.open(splits[0]); + + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + + // validate first row + assertNotNull(row); + assertEquals(4, row.getArity()); + // check binary + assertArrayEquals(new byte[]{0, 1, 2, 3, 4}, (byte[]) row.getField(0)); + // check boolean + assertEquals(false, row.getField(1)); + // check list + assertTrue(row.getField(2) instanceof Object[]); + Object[] list1 = (Object[]) row.getField(2); + assertEquals(2, list1.length); + assertEquals(Row.of(3, "good"), list1[0]); + assertEquals(Row.of(4, "bad"), list1[1]); + // check string + assertEquals("hi", row.getField(3)); + + // check that there is a second row with four fields + assertFalse(rowOrcInputFormat.reachedEnd()); + row = rowOrcInputFormat.nextRecord(null); + assertNotNull(row); + assertEquals(4, row.getArity()); + assertTrue(rowOrcInputFormat.reachedEnd()); + } + + @Test + public void testReadFileInSplits() throws IOException{ + + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + rowOrcInputFormat.selectFields(0, 1); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(4); + assertEquals(4, splits.length); + rowOrcInputFormat.openInputFormat(); + + long cnt = 0; + // read all splits + for (FileInputSplit split : splits) { + + // open split + rowOrcInputFormat.open(split); + // read and count all rows + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + } + // check that all rows have been read + assertEquals(1920800, cnt); + } + + @Test + public void testReadFileWithFilter() throws IOException{ + + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + rowOrcInputFormat.selectFields(0, 1); + + // read head and tail of file + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Or( + new OrcRowInputFormat.LessThan("_col0", PredicateLeaf.Type.LONG, 10L), + new OrcRowInputFormat.Not( + new OrcRowInputFormat.LessThanEquals("_col0", PredicateLeaf.Type.LONG, 1920000L)) + )); + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.Equals("_col1", PredicateLeaf.Type.STRING, "M")); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + + // open split + rowOrcInputFormat.open(splits[0]); + + // read and count all rows + long cnt = 0; + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + // check that only the first and last stripes of the file have been read. + // Each stripe has 5000 rows, except the last which has 800 rows. + assertEquals(5800, cnt); + } + + @Test + public void testReadFileWithEvolvedSchema() throws IOException{ + + rowOrcInputFormat = new OrcRowInputFormat( + getPath(TEST_FILE_FLAT), + "struct<_col0:int,_col1:string,_col4:string,_col3:string>", // previous version of schema + new Configuration()); + rowOrcInputFormat.selectFields(3, 0, 2); + + rowOrcInputFormat.addPredicate( + new OrcRowInputFormat.LessThan("_col0", PredicateLeaf.Type.LONG, 10L)); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(1); + assertEquals(1, splits.length); + rowOrcInputFormat.openInputFormat(); + + // open split + rowOrcInputFormat.open(splits[0]); + + // read and validate first row + assertFalse(rowOrcInputFormat.reachedEnd()); + Row row = rowOrcInputFormat.nextRecord(null); + assertNotNull(row); + assertEquals(3, row.getArity()); + assertEquals("Primary", row.getField(0)); + assertEquals(1, row.getField(1)); + assertEquals("M", row.getField(2)); + + // read and count remaining rows + long cnt = 1; + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + // check that only the first and last stripes of the file have been read. + // Each stripe has 5000 rows, except the last which has 800 rows. + assertEquals(5000, cnt); + } + + private String getPath(String fileName) { + return getClass().getClassLoader().getResource(fileName).getPath(); + } +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java index 3de6ab31010a5..e6ef1e16f8697 100644 --- a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceITCase.java @@ -18,125 +18,101 @@ package org.apache.flink.orc; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.typeutils.MapTypeInfo; -import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.java.BatchTableEnvironment; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.types.Row; -import org.hamcrest.CoreMatchers; -import org.junit.Assert; import org.junit.Test; -import java.net.URL; -import java.util.ArrayList; import java.util.List; +import static org.junit.Assert.assertEquals; + /** * Tests for {@link OrcTableSource}. */ public class OrcTableSourceITCase extends MultipleProgramsTestBase { - private static final String TEST1_SCHEMA = "struct>>," + - "list:array>," + - "map:map>>"; - - private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); - - - private static final String[] TEST1_DATA = new String[] { - "false,1,1024,65536,9223372036854775807,1.0,-15.0,[0, 1, 2, 3, 4],hi,[1,bye, 2,sigh],[3,good, 4,bad],{}", - "true,100,2048,65536,9223372036854775807,2.0,-5.0,[],bye,[1,bye, 2,sigh]," + - "[100000000,cat, -100000,in, 1234,hat],{chani=5,chani, mauddib=1,mauddib}" }; + private static final String TEST_FILE_FLAT = "test-data-flat.orc"; + private static final String TEST_SCHEMA_FLAT = + "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int,_col5:string,_col6:int,_col7:int,_col8:int>"; public OrcTableSourceITCase() { super(TestExecutionMode.COLLECTION); } @Test - public void testOrcTableSource() throws Exception { + public void testFullScan() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); - assert (test1URL != null); - OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); - - tEnv.registerTableSource("orcTable", orc); - - String query = "Select * from orcTable"; - Table t = tEnv.sql(query); + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_FLAT)) + .forOrcSchema(TEST_SCHEMA_FLAT) + .build(); + tEnv.registerTableSource("OrcTable", orc); + + String query = + "SELECT COUNT(*), " + + "MIN(_col0), MAX(_col0), " + + "MIN(_col1), MAX(_col1), " + + "MIN(_col2), MAX(_col2), " + + "MIN(_col3), MAX(_col3), " + + "MIN(_col4), MAX(_col4), " + + "MIN(_col5), MAX(_col5), " + + "MIN(_col6), MAX(_col6), " + + "MIN(_col7), MAX(_col7), " + + "MIN(_col8), MAX(_col8) " + + "FROM OrcTable"; + Table t = tEnv.sqlQuery(query); DataSet dataSet = tEnv.toDataSet(t, Row.class); - List records = dataSet.collect(); - - Assert.assertEquals(records.size(), 2); + List result = dataSet.collect(); - List actualRecords = new ArrayList<>(); - for (Row record : records) { - Assert.assertEquals(record.getArity(), 12); - actualRecords.add(record.toString()); - } - - Assert.assertThat(actualRecords, CoreMatchers.hasItems(TEST1_DATA)); + assertEquals(1, result.size()); + assertEquals( + "1920800,1,1920800,F,M,D,W,2 yr Degree,Unknown,500,10000,Good,Unknown,0,6,0,6,0,6", + result.get(0).toString()); } @Test - public void testOrcTableProjection() throws Exception { + public void testScanWithProjectionAndFilter() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); - assert(test1URL != null); - OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); - - tEnv.registerTableSource("orcTable", orc); - - String query = "Select middle,list,map from orcTable"; - Table t = tEnv.sql(query); - - String[] colNames = new String[] {"middle", "list", "map"}; - - RowTypeInfo rowTypeInfo = new RowTypeInfo( - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO}, - new String[] {"int1", "string1"}); - - RowTypeInfo structTypeInfo = new RowTypeInfo( - new TypeInformation[] {ObjectArrayTypeInfo.getInfoFor(rowTypeInfo)}, - new String[] {"list"}); - - TypeInformation[] colTypes = new TypeInformation[] { - structTypeInfo, - ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), - new MapTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, rowTypeInfo) - }; - - TableSchema actualTableSchema = new TableSchema(colNames, colTypes); - - Assert.assertArrayEquals(t.getSchema().getColumnNames(), colNames); - Assert.assertArrayEquals(t.getSchema().getTypes(), colTypes); - Assert.assertEquals(actualTableSchema.toString(), t.getSchema().toString()); + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_FLAT)) + .forOrcSchema(TEST_SCHEMA_FLAT) + .build(); + tEnv.registerTableSource("OrcTable", orc); + + String query = + "SELECT " + + "MIN(_col4), MAX(_col4), " + + "MIN(_col3), MAX(_col3), " + + "MIN(_col0), MAX(_col0), " + + "MIN(_col2), MAX(_col2), " + + "COUNT(*) " + + "FROM OrcTable " + + "WHERE (_col0 BETWEEN 4975 and 5024 OR _col0 BETWEEN 9975 AND 10024) AND _col1 = 'F'"; + Table t = tEnv.sqlQuery(query); DataSet dataSet = tEnv.toDataSet(t, Row.class); - List records = dataSet.collect(); - - Assert.assertEquals(records.size(), 2); - for (Row record: records) { - Assert.assertEquals(record.getArity(), 3); - } + List result = dataSet.collect(); + assertEquals(1, result.size()); + assertEquals( + "1500,6000,2 yr Degree,Unknown,4976,10024,D,W,50", + result.get(0).toString()); } + private String getPath(String fileName) { + return getClass().getClassLoader().getResource(fileName).getPath(); + } } diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java index c285054c6e155..4e4be772119e7 100644 --- a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java @@ -18,96 +18,248 @@ package org.apache.flink.orc; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.typeutils.MapTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.java.BatchTableEnvironment; +import org.apache.flink.table.expressions.EqualTo; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.GetCompositeField; +import org.apache.flink.table.expressions.GreaterThan; +import org.apache.flink.table.expressions.ItemAt; +import org.apache.flink.table.expressions.Literal; +import org.apache.flink.table.expressions.ResolvedFieldReference; +import org.apache.flink.types.Row; -import org.junit.Assert; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; import org.junit.Test; +import org.mockito.ArgumentCaptor; -import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; /** * Unit Tests for {@link OrcTableSource}. */ public class OrcTableSourceTest { - private static final String TEST1_SCHEMA = "struct>>," + - "list:array>," + - "map:map>>"; + private static final String TEST_FILE_NESTED = "test-data-nested.orc"; + private static final String TEST_SCHEMA_NESTED = + "struct<" + + "boolean1:boolean," + + "byte1:tinyint," + + "short1:smallint," + + "int1:int," + + "long1:bigint," + + "float1:float," + + "double1:double," + + "bytes1:binary," + + "string1:string," + + "middle:struct<" + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">," + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">," + + "map:map<" + + "string," + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">"; + + @Test + public void testGetReturnType() throws Exception { + + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_NESTED)) + .forOrcSchema(TEST_SCHEMA_NESTED) + .build(); - private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); + TypeInformation returnType = orc.getReturnType(); + assertNotNull(returnType); + assertTrue(returnType instanceof RowTypeInfo); + RowTypeInfo rowType = (RowTypeInfo) returnType; + + RowTypeInfo expected = Types.ROW_NAMED(getNestedFieldNames(), getNestedFieldTypes()); + assertEquals(expected, rowType); + } @Test - public void testOrcSchema() throws Exception { + public void testGetTableSchema() throws Exception { + + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_NESTED)) + .forOrcSchema(TEST_SCHEMA_NESTED) + .build(); + + TableSchema schema = orc.getTableSchema(); + assertNotNull(schema); + assertArrayEquals(getNestedFieldNames(), schema.getColumnNames()); + assertArrayEquals(getNestedFieldTypes(), schema.getTypes()); + } + + @Test + public void testProjectFields() throws Exception { + + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_NESTED)) + .forOrcSchema(TEST_SCHEMA_NESTED) + .build(); + + OrcTableSource projected = (OrcTableSource) orc.projectFields(new int[]{3, 5, 1, 0}); - assert(test1URL != null); - OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + // ensure copy is returned + assertTrue(orc != projected); - String expectedSchema = "Row(boolean1: Boolean, byte1: Byte, short1: Short, int1: Integer, long1: Long, " + - "float1: Float, double1: Double, bytes1: byte[], string1: String, " + - "middle: Row(list: ObjectArrayTypeInfo), " + - "list: ObjectArrayTypeInfo, " + - "map: Map)"; + // ensure table schema is identical + assertEquals(orc.getTableSchema(), projected.getTableSchema()); - Assert.assertEquals(expectedSchema, orc.getReturnType().toString()); + // ensure return type was adapted + String[] fieldNames = getNestedFieldNames(); + TypeInformation[] fieldTypes = getNestedFieldTypes(); + assertEquals( + Types.ROW_NAMED( + new String[] {fieldNames[3], fieldNames[5], fieldNames[1], fieldNames[0]}, + new TypeInformation[] {fieldTypes[3], fieldTypes[5], fieldTypes[1], fieldTypes[0]}), + projected.getReturnType()); + // ensure IF is configured with selected fields + OrcTableSource spyTS = spy(projected); + OrcRowInputFormat mockIF = mock(OrcRowInputFormat.class); + doReturn(mockIF).when(spyTS).buildOrcInputFormat(); + spyTS.getDataSet(mock(ExecutionEnvironment.class)); + verify(mockIF).selectFields(eq(3), eq(5), eq(1), eq(0)); } @Test - public void testOrcTableSchema() throws Exception { + public void testApplyPredicate() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_NESTED)) + .forOrcSchema(TEST_SCHEMA_NESTED) + .build(); - assert(test1URL != null); - OrcTableSource orc = new OrcTableSource(test1URL.getPath(), TEST1_SCHEMA); + // expressions for predicates + Expression pred1 = new GreaterThan( + new ResolvedFieldReference("int1", Types.INT), + new Literal(100, Types.INT)); + Expression pred2 = new EqualTo( + new ResolvedFieldReference("string1", Types.STRING), + new Literal("hello", Types.STRING)); + Expression pred3 = new EqualTo( + new GetCompositeField( + new ItemAt( + new ResolvedFieldReference( + "list", + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[] {"int1", "string1"}, Types.INT, Types.STRING))), + new Literal(1, Types.INT)), + "int1"), + new Literal(1, Types.INT) + ); + ArrayList preds = new ArrayList<>(); + preds.add(pred1); + preds.add(pred2); + preds.add(pred3); - tEnv.registerTableSource("orcTable", orc); - String query = "Select * from orcTable"; - Table t = tEnv.sql(query); + // apply predicates on TableSource + OrcTableSource projected = (OrcTableSource) orc.applyPredicate(preds); - String[] colNames = new String[] { - "boolean1", "byte1", "short1", "int1", "long1", "float1", - "double1", "bytes1", "string1", "list", "list0", "map" - }; + // ensure copy is returned + assertTrue(orc != projected); - RowTypeInfo rowTypeInfo = new RowTypeInfo( - new TypeInformation[] { - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO}, - new String[] {"int1", "string1"}); - - TypeInformation[] colTypes = new TypeInformation[] { - BasicTypeInfo.BOOLEAN_TYPE_INFO, - BasicTypeInfo.BYTE_TYPE_INFO, - BasicTypeInfo.SHORT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.FLOAT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), - ObjectArrayTypeInfo.getInfoFor(rowTypeInfo), - new MapTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, rowTypeInfo) - }; - TableSchema expectedTableSchema = new TableSchema(colNames, colTypes); + // ensure table schema is identical + assertEquals(orc.getTableSchema(), projected.getTableSchema()); + + // ensure return type is identical + assertEquals( + Types.ROW_NAMED(getNestedFieldNames(), getNestedFieldTypes()), + projected.getReturnType()); + + // ensure IF is configured with supported predicates + OrcTableSource spyTS = spy(projected); + OrcRowInputFormat mockIF = mock(OrcRowInputFormat.class); + doReturn(mockIF).when(spyTS).buildOrcInputFormat(); + spyTS.getDataSet(mock(ExecutionEnvironment.class)); + + ArgumentCaptor arguments = ArgumentCaptor.forClass(OrcRowInputFormat.Predicate.class); + verify(mockIF, times(2)).addPredicate(arguments.capture()); + List values = arguments.getAllValues().stream().map(Object::toString).collect(Collectors.toList()); + assertTrue(values.contains( + new OrcRowInputFormat.Not(new OrcRowInputFormat.LessThanEquals("int1", PredicateLeaf.Type.LONG, 100)).toString())); + assertTrue(values.contains( + new OrcRowInputFormat.Equals("string1", PredicateLeaf.Type.STRING, "hello").toString())); + + // ensure filter pushdown is correct + assertTrue(spyTS.isFilterPushedDown()); + assertFalse(orc.isFilterPushedDown()); + } + + private String getPath(String fileName) { + return getClass().getClassLoader().getResource(fileName).getPath(); + } - Assert.assertArrayEquals(t.getSchema().getColumnNames(), colNames); - Assert.assertArrayEquals(t.getSchema().getTypes(), colTypes); - Assert.assertEquals(expectedTableSchema.toString(), t.getSchema().toString()); + private String[] getNestedFieldNames() { + return new String[] { + "boolean1", "byte1", "short1", "int1", "long1", "float1", "double1", "bytes1", "string1", "middle", "list", "map" + }; + } + private TypeInformation[] getNestedFieldTypes() { + return new TypeInformation[]{ + Types.BOOLEAN, Types.BYTE, Types.SHORT, Types.INT, Types.LONG, Types.FLOAT, Types.DOUBLE, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, Types.STRING, + Types.ROW_NAMED( + new String[]{"list"}, + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ) + ), + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ), + new MapTypeInfo<>( + Types.STRING, + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ) + }; } } diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcUtilsTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcUtilsTest.java new file mode 100644 index 0000000000000..2cb17150bf306 --- /dev/null +++ b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/OrcUtilsTest.java @@ -0,0 +1,148 @@ +/* + * 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.flink.orc; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import org.apache.orc.TypeDescription; +import org.junit.Assert; +import org.junit.Test; + +/** + * Unit tests for {@link OrcUtils}. + * + */ +public class OrcUtilsTest { + + @Test + public void testFlatSchemaToTypeInfo1() { + + String schema = + "struct<" + + "boolean1:boolean," + + "byte1:tinyint," + + "short1:smallint," + + "int1:int," + + "long1:bigint," + + "float1:float," + + "double1:double," + + "bytes1:binary," + + "string1:string," + + "date1:date," + + "timestamp1:timestamp," + + "decimal1:decimal(5,2)" + + ">"; + TypeInformation typeInfo = OrcUtils.schemaToTypeInfo(TypeDescription.fromString(schema)); + + Assert.assertNotNull(typeInfo); + Assert.assertTrue(typeInfo instanceof RowTypeInfo); + RowTypeInfo rowTypeInfo = (RowTypeInfo) typeInfo; + + // validate field types + Assert.assertArrayEquals( + new TypeInformation[]{ + Types.BOOLEAN, Types.BYTE, Types.SHORT, Types.INT, Types.LONG, Types.FLOAT, Types.DOUBLE, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, Types.STRING, + Types.SQL_DATE, Types.SQL_TIMESTAMP, BasicTypeInfo.BIG_DEC_TYPE_INFO + }, + rowTypeInfo.getFieldTypes()); + + // validate field names + Assert.assertArrayEquals( + new String[] { + "boolean1", "byte1", "short1", "int1", "long1", "float1", "double1", + "bytes1", "string1", "date1", "timestamp1", "decimal1" + }, + rowTypeInfo.getFieldNames()); + + } + + @Test + public void testNestedSchemaToTypeInfo1() { + + String schema = + "struct<" + + "middle:struct<" + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">," + + "list:array<" + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">," + + "map:map<" + + "string," + + "struct<" + + "int1:int," + + "string1:string" + + ">" + + ">" + + ">"; + TypeInformation typeInfo = OrcUtils.schemaToTypeInfo(TypeDescription.fromString(schema)); + + Assert.assertNotNull(typeInfo); + Assert.assertTrue(typeInfo instanceof RowTypeInfo); + RowTypeInfo rowTypeInfo = (RowTypeInfo) typeInfo; + + // validate field types + Assert.assertArrayEquals( + new TypeInformation[]{ + Types.ROW_NAMED( + new String[]{"list"}, + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ) + ), + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ), + new MapTypeInfo<>( + Types.STRING, + Types.ROW_NAMED( + new String[]{"int1", "string1"}, + Types.INT, Types.STRING + ) + ) + }, + rowTypeInfo.getFieldTypes()); + + // validate field names + Assert.assertArrayEquals( + new String[] {"middle", "list", "map"}, + rowTypeInfo.getFieldNames()); + } +} diff --git a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java b/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java deleted file mode 100644 index 60008a0838612..0000000000000 --- a/flink-connectors/flink-orc/src/test/java/org/apache/flink/orc/RowOrcInputFormatTest.java +++ /dev/null @@ -1,472 +0,0 @@ -/* - * 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.flink.orc; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.types.Row; - -import org.apache.hadoop.conf.Configuration; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -/** - * Tests for the {@link RowOrcInputFormat}. - */ - -public class RowOrcInputFormatTest { - - private RowOrcInputFormat rowOrcInputFormat; - - @After - public void tearDown() throws IOException { - if (rowOrcInputFormat != null) { - rowOrcInputFormat.close(); - rowOrcInputFormat.closeInputFormat(); - } - rowOrcInputFormat = null; - } - - private final URL test1URL = getClass().getClassLoader().getResource("TestOrcFile.test1.orc"); - - private static final String TEST1_SCHEMA = "struct>>," + - "list:array>," + - "map:map>>"; - - private static final String[] TEST1_DATA = new String[] { - "false,1,1024,65536,9223372036854775807,1.0,-15.0,[0, 1, 2, 3, 4],hi,[1,bye, 2,sigh],[3,good, 4,bad],{}", - "true,100,2048,65536,9223372036854775807,2.0,-5.0,[],bye,[1,bye, 2,sigh]," + - "[100000000,cat, -100000,in, 1234,hat],{chani=5,chani, mauddib=1,mauddib}" }; - - private static final String[] TEST1_PROJECTED_DATA = new String[] { - "{},[3,good, 4,bad],[1,bye, 2,sigh],hi,[0, 1, 2, 3, 4],-15.0,1.0,9223372036854775807,65536,1024,1,false", - "{chani=5,chani, mauddib=1,mauddib},[100000000,cat, -100000,in, 1234,hat],[1,bye, 2,sigh],bye," + - "[],-5.0,2.0,9223372036854775807,65536,2048,100,true" }; - - private static final String TEST1_INVALID_SCHEMA = "struct>>," + - "list:array>," + - "map:map>>"; - - @Test(expected = FileNotFoundException.class) - public void testInvalidPath() throws IOException{ - - rowOrcInputFormat = new RowOrcInputFormat("TestOrcFile.test2.orc", TEST1_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - rowOrcInputFormat.open(inputSplits[0]); - - } - - @Test(expected = RuntimeException.class) - public void testInvalidSchema() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_INVALID_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - rowOrcInputFormat.open(inputSplits[0]); - - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testInvalidProjection() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - int[] projectionMask = {14}; - rowOrcInputFormat.setFieldMapping(projectionMask); - } - - @Test - public void testMajorDataTypes() throws IOException{ - - // test for boolean,byte,short,int,long,float,double,bytes,string,struct,list,map - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.toString(), TEST1_DATA[count++]); - } - } - } - - @Test - public void testProjection() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - int[] projectionMask = {11, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0}; - rowOrcInputFormat.setFieldMapping(projectionMask); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.toString(), TEST1_PROJECTED_DATA[count++]); - } - } - - } - - @Test - public void testTimeStampAndDate() throws IOException{ - - URL expectedDataURL = getClass().getClassLoader().getResource("TestOrcFile.testDate1900.dat"); - assert(expectedDataURL != null); - List expectedTimeStampAndDate = Files.readAllLines(Paths.get(expectedDataURL.getPath())); - - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.testDate1900.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct"; - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - List actualTimeStampAndDate = new ArrayList<>(); - - Row row = null; - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - count++; - if (count <= 10000) { - actualTimeStampAndDate.add(row.getField(0) + "," + row.getField(1)); - } - - } - } - Assert.assertEquals(count, 70000); - Assert.assertEquals(expectedTimeStampAndDate.size(), actualTimeStampAndDate.size()); - Assert.assertEquals(expectedTimeStampAndDate.toString(), actualTimeStampAndDate.toString()); - - } - - @Test - public void testDecimal() throws IOException{ - - URL expectedDataURL = getClass().getClassLoader().getResource("decimal.dat"); - List expectedDecimal = Files.readAllLines(Paths.get(expectedDataURL.getPath())); - - URL testInputURL = getClass().getClassLoader().getResource("decimal.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct<_col0:decimal(10,5)>"; - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - List actualDecimal = new ArrayList<>(); - - Row row = null; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - actualDecimal.add(row.getField(0)); - } - } - - Assert.assertEquals(expectedDecimal.size(), actualDecimal.size()); - Assert.assertEquals(expectedDecimal.toString(), actualDecimal.toString()); - - } - - @Test - public void testEmptyFile() throws IOException{ - - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.emptyFile.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - - rowOrcInputFormat = new RowOrcInputFormat(path, TEST1_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = new Row(1); - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - count++; - } - } - - Assert.assertEquals(count, 0); - } - - @Test - public void testLargeFile() throws IOException{ - - URL testInputURL = getClass().getClassLoader().getResource("demo-11-none.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int," + - "_col5:string,_col6:int,_col7:int,_col8:int>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = new Row(1); - int count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - count++; - } - } - - Assert.assertEquals(count, 1920800); - } - - @Test - public void testProducedType() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - rowOrcInputFormat.open(inputSplits[0]); - - TypeInformation type = rowOrcInputFormat.getProducedType(); - Assert.assertEquals(type.toString(), "Row(boolean1: Boolean, byte1: Byte, short1: Short, int1: Integer," + - " long1: Long, float1: Float, double1: Double, bytes1: byte[], string1: String," + - " middle: Row(list: ObjectArrayTypeInfo)," + - " list: ObjectArrayTypeInfo," + - " map: Map)"); - - } - - @Test - public void testProducedTypeWithProjection() throws IOException{ - - assert(test1URL != null); - rowOrcInputFormat = new RowOrcInputFormat(test1URL.getPath(), TEST1_SCHEMA, new Configuration()); - int[] projectionMask = {9, 10, 11}; - rowOrcInputFormat.setFieldMapping(projectionMask); - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - rowOrcInputFormat.open(inputSplits[0]); - - TypeInformation type = rowOrcInputFormat.getProducedType(); - Assert.assertEquals(type.toString(), "Row(middle: Row(list: ObjectArrayTypeInfo)," + - " list: ObjectArrayTypeInfo," + - " map: Map)"); - - } - - @Test - public void testLongList() throws Exception { - - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.listlong.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - long count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.getArity(), 1); - Object object = row.getField(0); - long[] l = (long[]) object; - - Assert.assertEquals(l.length, 2); - if (count < 50) { - Assert.assertArrayEquals(l, new long[]{count, count + 1}); - } - else { - Assert.assertArrayEquals(l, new long[]{0L, 0L}); - } - count = count + 2; - } - } - Assert.assertEquals(count, 100); - } - - @Test - public void testStringList() throws Exception { - - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.liststring.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - long count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.getArity(), 1); - Object object = row.getField(0); - String[] l = (String[]) object; - - Assert.assertEquals(l.length, 2); - Assert.assertArrayEquals(l, new String[]{"hello" + count, "hello" + (count + 1) }); - count = count + 2; - } - } - Assert.assertEquals(count, 200); - } - - @Test - public void testListOfListOfStructOfLong() throws Exception { - URL testInputURL = getClass().getClassLoader().getResource("TestOrcFile.listliststructlong.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct>>>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - - rowOrcInputFormat.openInputFormat(); - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(1); - - Assert.assertEquals(inputSplits.length, 1); - - Row row = null; - long count = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - while (!rowOrcInputFormat.reachedEnd()) { - - row = rowOrcInputFormat.nextRecord(row); - Assert.assertEquals(row.getArity(), 1); - - Object[] objects = (Object[]) row.getField(0); - Assert.assertEquals(objects.length, 1); - - Object[] objects1 = (Object[]) objects[0]; - Assert.assertEquals(objects1.length, 1); - - Row[] nestedRows = Arrays.copyOf(objects1, objects1.length, Row[].class); - Assert.assertEquals(nestedRows.length, 1); - - Assert.assertEquals(nestedRows[0].getArity(), 1); - - Assert.assertEquals(nestedRows[0].getField(0), count); - - count++; - } - } - Assert.assertEquals(count, 100); - } - - @Test - public void testSplit() throws IOException{ - - URL testInputURL = getClass().getClassLoader().getResource("demo-11-none.orc"); - assert(testInputURL != null); - String path = testInputURL.getPath(); - String schema = "struct<_col0:int,_col1:string,_col2:string,_col3:string,_col4:int," + - "_col5:string,_col6:int,_col7:int,_col8:int>"; - - rowOrcInputFormat = new RowOrcInputFormat(path, schema, new Configuration()); - rowOrcInputFormat.openInputFormat(); - - FileInputSplit[] inputSplits = rowOrcInputFormat.createInputSplits(10); - - Assert.assertEquals(inputSplits.length, 10); - - Row row = null; - int countTotalRecords = 0; - for (FileInputSplit split : inputSplits) { - rowOrcInputFormat.open(split); - int countSplitRecords = 0; - while (!rowOrcInputFormat.reachedEnd()) { - row = rowOrcInputFormat.nextRecord(row); - countSplitRecords++; - } - Assert.assertNotEquals(countSplitRecords, 1920800); - countTotalRecords += countSplitRecords; - } - - Assert.assertEquals(countTotalRecords, 1920800); - } - -} diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.emptyFile.orc deleted file mode 100644 index ecdadcbff134615d7eefcb740d55fe710cee059b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 523 zcmaix$xg#C5Qe>EnJmx{S{5I3LY$gwPl(Ki0VfCx86O*nAn33yj{0RALJfJ%g= z-yY9=v&depmn0*ZJW?2kTu94yT=M+Mv7+18fd!+++rKoYVb8*(6m+$6w2JQ%VP zoBWuE$qYVmaJZY`s{`<^gga2tKpn+nltV!*rn5eyFL-h~g3j5(B6iCb91v JV7f4AZ$rdMGkni%gxIAM^?8)@NzR1G03_ zQ~I=$l8)`w(zKm!g}HebqmEB z$(}8Vks5ovjmb^kX-N*YbktC-?e0-&YOo*RHb1o4jT#G6Uq?2aIh-Ryw8c%=ROlA(7bVdZ!>jqI83wO`#jQ!BZ1BcvtYw$6ASzM8ou-0iYb#i<<5?Y8 zcR86{`PbuAyu-G0%+|g?R_Q}}d}YnIwm1mjJj`x;n`S?P$jk3QlgcznjWaUDr?(61 z3Bbqtmr$)JCquHRII7bMwFr!QAr#AIzr0d}oOz2A#o{B}df>n)gR+|a0pW23?LA#& zcAvMyU=0E<{s9R90KhQ<01*HHz%c*h0OOM8V&O0fV1Sam z94SKV91H?U5|e}j7yy(20OMd5_{=QD&A}|NfLTIF!A`?SiI0Ovh>=l>J-0F^v$(`i ziIsy(h>=N&iG#(!$clx-D1ZS<@^YjIv2!p8BuPvX4q$M2!43cjO%MSXfFL3Q01PM` NFb0D30|QS|Llz^%2`T^p diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.liststring.orc deleted file mode 100644 index 75a5f2a10a3fe703394fd3700e56d06149f2785b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1298 zcmV+t1?~DzQbQ2{015^Y1PEkM04M+e|ol`QeugC00RIb1OOdU!HN?>5PjWgV`Yg@6M_T} zQj^_<=xJ8=;;j=RGjzO2ys!ex3fVHFMPf}N0~_|J$b(oPe=BEqIe z@P+Q{3BlH(q^KS_Q66s*qbnS;<%}RL3%{1=BQ6o&kDZt%K`b7imB4z^u%)5M`$3|1 z!$5h0e8-8HVU#(PcXGQlE7Kax-Uj?& z!#wOIPOfEmByAH>txpV$-a8&5P(_YG26PHYo!MV84}7~>7RQl&sAf< zk-72I{bY7VtVnN>nrTZf<~#-2&t3rAgjOZ}g{>kyvM5fFwQb86006*a0stBV z06o(|uADFw1<@^07-PJkNPiG1`=3RjicH>&qi(4C^XvETKgZ9%%X`oFj`v>go$tNh zyWab}cfXe&_Gx?^pGN=%B+x(vl~d*HH>jK{r^=~vs+=mP%BeC`hRW!#s0@{%GE|1j zP#G$h%B6CtT)h#MOXX6zR4$cEWvWb-sWMe&FIHu$OqHoJRc@7AZk4<5qjIa< zD!0l~St?6qsVtSHvii&_OJ%7%Dv!#e@~Av2kIJL+REx@^vQ@UqR@o|BWvgtJt+G|N z3Wp4b4u=qj5{DFr7Ka#z8iyQ5-v5mAzL39?zm&h0znH(8zntrtf~KN5`+YS{QB&2F zHFZs4Q`wX@wM}s|`oHB(eN*66I3-SvQ{+@RWlo)Q_5LcIQm57_cB-9nr`{=eDxQ*O z_C1Q8s;BI!dkUY*r}U|Pil6G|?)%n14S*Iv6QB*y2xtW~1KI%%fmJU}fwn+npf%7O zXb&_9S_DmkHo;SWt%7DjyP#pvGH4pK4H^fngXY2Be*qN$00#!ZSqA{XV+a6%001?K zK?;CC36VJHcrhQ)_j>Tb#)6j#Gf5Ahjca;@_Q=nTv)p)8Qgd?h4TRXiw56rQ67c{A0IUE2 z<6su}%^}6Z!7MP5RbmOVg0n`75+4VT5F?`$dv0Y;W^sw35-SIn5F?Wk69>CNiWLh- zN&o|tl;$|Ws3gM0mXVs1lW!oz4yG+FC6FgUzm2LK3J5CIr~AR+<)3@9fs27>eh I15Z*z7I1J@L;wH) diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat b/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat deleted file mode 100644 index 59b933ddf0b5c..0000000000000 --- a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.dat +++ /dev/null @@ -1,10000 +0,0 @@ -1900-05-05 12:34:56.1,1900-12-25 -1900-05-05 12:34:56.1001,1900-12-25 -1900-05-05 12:34:56.1002,1900-12-25 -1900-05-05 12:34:56.1003,1900-12-25 -1900-05-05 12:34:56.1004,1900-12-25 -1900-05-05 12:34:56.1005,1900-12-25 -1900-05-05 12:34:56.1006,1900-12-25 -1900-05-05 12:34:56.1007,1900-12-25 -1900-05-05 12:34:56.1008,1900-12-25 -1900-05-05 12:34:56.1009,1900-12-25 -1900-05-05 12:34:56.101,1900-12-25 -1900-05-05 12:34:56.1011,1900-12-25 -1900-05-05 12:34:56.1012,1900-12-25 -1900-05-05 12:34:56.1013,1900-12-25 -1900-05-05 12:34:56.1014,1900-12-25 -1900-05-05 12:34:56.1015,1900-12-25 -1900-05-05 12:34:56.1016,1900-12-25 -1900-05-05 12:34:56.1017,1900-12-25 -1900-05-05 12:34:56.1018,1900-12-25 -1900-05-05 12:34:56.1019,1900-12-25 -1900-05-05 12:34:56.102,1900-12-25 -1900-05-05 12:34:56.1021,1900-12-25 -1900-05-05 12:34:56.1022,1900-12-25 -1900-05-05 12:34:56.1023,1900-12-25 -1900-05-05 12:34:56.1024,1900-12-25 -1900-05-05 12:34:56.1025,1900-12-25 -1900-05-05 12:34:56.1026,1900-12-25 -1900-05-05 12:34:56.1027,1900-12-25 -1900-05-05 12:34:56.1028,1900-12-25 -1900-05-05 12:34:56.1029,1900-12-25 -1900-05-05 12:34:56.103,1900-12-25 -1900-05-05 12:34:56.1031,1900-12-25 -1900-05-05 12:34:56.1032,1900-12-25 -1900-05-05 12:34:56.1033,1900-12-25 -1900-05-05 12:34:56.1034,1900-12-25 -1900-05-05 12:34:56.1035,1900-12-25 -1900-05-05 12:34:56.1036,1900-12-25 -1900-05-05 12:34:56.1037,1900-12-25 -1900-05-05 12:34:56.1038,1900-12-25 -1900-05-05 12:34:56.1039,1900-12-25 -1900-05-05 12:34:56.104,1900-12-25 -1900-05-05 12:34:56.1041,1900-12-25 -1900-05-05 12:34:56.1042,1900-12-25 -1900-05-05 12:34:56.1043,1900-12-25 -1900-05-05 12:34:56.1044,1900-12-25 -1900-05-05 12:34:56.1045,1900-12-25 -1900-05-05 12:34:56.1046,1900-12-25 -1900-05-05 12:34:56.1047,1900-12-25 -1900-05-05 12:34:56.1048,1900-12-25 -1900-05-05 12:34:56.1049,1900-12-25 -1900-05-05 12:34:56.105,1900-12-25 -1900-05-05 12:34:56.1051,1900-12-25 -1900-05-05 12:34:56.1052,1900-12-25 -1900-05-05 12:34:56.1053,1900-12-25 -1900-05-05 12:34:56.1054,1900-12-25 -1900-05-05 12:34:56.1055,1900-12-25 -1900-05-05 12:34:56.1056,1900-12-25 -1900-05-05 12:34:56.1057,1900-12-25 -1900-05-05 12:34:56.1058,1900-12-25 -1900-05-05 12:34:56.1059,1900-12-25 -1900-05-05 12:34:56.106,1900-12-25 -1900-05-05 12:34:56.1061,1900-12-25 -1900-05-05 12:34:56.1062,1900-12-25 -1900-05-05 12:34:56.1063,1900-12-25 -1900-05-05 12:34:56.1064,1900-12-25 -1900-05-05 12:34:56.1065,1900-12-25 -1900-05-05 12:34:56.1066,1900-12-25 -1900-05-05 12:34:56.1067,1900-12-25 -1900-05-05 12:34:56.1068,1900-12-25 -1900-05-05 12:34:56.1069,1900-12-25 -1900-05-05 12:34:56.107,1900-12-25 -1900-05-05 12:34:56.1071,1900-12-25 -1900-05-05 12:34:56.1072,1900-12-25 -1900-05-05 12:34:56.1073,1900-12-25 -1900-05-05 12:34:56.1074,1900-12-25 -1900-05-05 12:34:56.1075,1900-12-25 -1900-05-05 12:34:56.1076,1900-12-25 -1900-05-05 12:34:56.1077,1900-12-25 -1900-05-05 12:34:56.1078,1900-12-25 -1900-05-05 12:34:56.1079,1900-12-25 -1900-05-05 12:34:56.108,1900-12-25 -1900-05-05 12:34:56.1081,1900-12-25 -1900-05-05 12:34:56.1082,1900-12-25 -1900-05-05 12:34:56.1083,1900-12-25 -1900-05-05 12:34:56.1084,1900-12-25 -1900-05-05 12:34:56.1085,1900-12-25 -1900-05-05 12:34:56.1086,1900-12-25 -1900-05-05 12:34:56.1087,1900-12-25 -1900-05-05 12:34:56.1088,1900-12-25 -1900-05-05 12:34:56.1089,1900-12-25 -1900-05-05 12:34:56.109,1900-12-25 -1900-05-05 12:34:56.1091,1900-12-25 -1900-05-05 12:34:56.1092,1900-12-25 -1900-05-05 12:34:56.1093,1900-12-25 -1900-05-05 12:34:56.1094,1900-12-25 -1900-05-05 12:34:56.1095,1900-12-25 -1900-05-05 12:34:56.1096,1900-12-25 -1900-05-05 12:34:56.1097,1900-12-25 -1900-05-05 12:34:56.1098,1900-12-25 -1900-05-05 12:34:56.1099,1900-12-25 -1900-05-05 12:34:56.11,1900-12-25 -1900-05-05 12:34:56.1101,1900-12-25 -1900-05-05 12:34:56.1102,1900-12-25 -1900-05-05 12:34:56.1103,1900-12-25 -1900-05-05 12:34:56.1104,1900-12-25 -1900-05-05 12:34:56.1105,1900-12-25 -1900-05-05 12:34:56.1106,1900-12-25 -1900-05-05 12:34:56.1107,1900-12-25 -1900-05-05 12:34:56.1108,1900-12-25 -1900-05-05 12:34:56.1109,1900-12-25 -1900-05-05 12:34:56.111,1900-12-25 -1900-05-05 12:34:56.1111,1900-12-25 -1900-05-05 12:34:56.1112,1900-12-25 -1900-05-05 12:34:56.1113,1900-12-25 -1900-05-05 12:34:56.1114,1900-12-25 -1900-05-05 12:34:56.1115,1900-12-25 -1900-05-05 12:34:56.1116,1900-12-25 -1900-05-05 12:34:56.1117,1900-12-25 -1900-05-05 12:34:56.1118,1900-12-25 -1900-05-05 12:34:56.1119,1900-12-25 -1900-05-05 12:34:56.112,1900-12-25 -1900-05-05 12:34:56.1121,1900-12-25 -1900-05-05 12:34:56.1122,1900-12-25 -1900-05-05 12:34:56.1123,1900-12-25 -1900-05-05 12:34:56.1124,1900-12-25 -1900-05-05 12:34:56.1125,1900-12-25 -1900-05-05 12:34:56.1126,1900-12-25 -1900-05-05 12:34:56.1127,1900-12-25 -1900-05-05 12:34:56.1128,1900-12-25 -1900-05-05 12:34:56.1129,1900-12-25 -1900-05-05 12:34:56.113,1900-12-25 -1900-05-05 12:34:56.1131,1900-12-25 -1900-05-05 12:34:56.1132,1900-12-25 -1900-05-05 12:34:56.1133,1900-12-25 -1900-05-05 12:34:56.1134,1900-12-25 -1900-05-05 12:34:56.1135,1900-12-25 -1900-05-05 12:34:56.1136,1900-12-25 -1900-05-05 12:34:56.1137,1900-12-25 -1900-05-05 12:34:56.1138,1900-12-25 -1900-05-05 12:34:56.1139,1900-12-25 -1900-05-05 12:34:56.114,1900-12-25 -1900-05-05 12:34:56.1141,1900-12-25 -1900-05-05 12:34:56.1142,1900-12-25 -1900-05-05 12:34:56.1143,1900-12-25 -1900-05-05 12:34:56.1144,1900-12-25 -1900-05-05 12:34:56.1145,1900-12-25 -1900-05-05 12:34:56.1146,1900-12-25 -1900-05-05 12:34:56.1147,1900-12-25 -1900-05-05 12:34:56.1148,1900-12-25 -1900-05-05 12:34:56.1149,1900-12-25 -1900-05-05 12:34:56.115,1900-12-25 -1900-05-05 12:34:56.1151,1900-12-25 -1900-05-05 12:34:56.1152,1900-12-25 -1900-05-05 12:34:56.1153,1900-12-25 -1900-05-05 12:34:56.1154,1900-12-25 -1900-05-05 12:34:56.1155,1900-12-25 -1900-05-05 12:34:56.1156,1900-12-25 -1900-05-05 12:34:56.1157,1900-12-25 -1900-05-05 12:34:56.1158,1900-12-25 -1900-05-05 12:34:56.1159,1900-12-25 -1900-05-05 12:34:56.116,1900-12-25 -1900-05-05 12:34:56.1161,1900-12-25 -1900-05-05 12:34:56.1162,1900-12-25 -1900-05-05 12:34:56.1163,1900-12-25 -1900-05-05 12:34:56.1164,1900-12-25 -1900-05-05 12:34:56.1165,1900-12-25 -1900-05-05 12:34:56.1166,1900-12-25 -1900-05-05 12:34:56.1167,1900-12-25 -1900-05-05 12:34:56.1168,1900-12-25 -1900-05-05 12:34:56.1169,1900-12-25 -1900-05-05 12:34:56.117,1900-12-25 -1900-05-05 12:34:56.1171,1900-12-25 -1900-05-05 12:34:56.1172,1900-12-25 -1900-05-05 12:34:56.1173,1900-12-25 -1900-05-05 12:34:56.1174,1900-12-25 -1900-05-05 12:34:56.1175,1900-12-25 -1900-05-05 12:34:56.1176,1900-12-25 -1900-05-05 12:34:56.1177,1900-12-25 -1900-05-05 12:34:56.1178,1900-12-25 -1900-05-05 12:34:56.1179,1900-12-25 -1900-05-05 12:34:56.118,1900-12-25 -1900-05-05 12:34:56.1181,1900-12-25 -1900-05-05 12:34:56.1182,1900-12-25 -1900-05-05 12:34:56.1183,1900-12-25 -1900-05-05 12:34:56.1184,1900-12-25 -1900-05-05 12:34:56.1185,1900-12-25 -1900-05-05 12:34:56.1186,1900-12-25 -1900-05-05 12:34:56.1187,1900-12-25 -1900-05-05 12:34:56.1188,1900-12-25 -1900-05-05 12:34:56.1189,1900-12-25 -1900-05-05 12:34:56.119,1900-12-25 -1900-05-05 12:34:56.1191,1900-12-25 -1900-05-05 12:34:56.1192,1900-12-25 -1900-05-05 12:34:56.1193,1900-12-25 -1900-05-05 12:34:56.1194,1900-12-25 -1900-05-05 12:34:56.1195,1900-12-25 -1900-05-05 12:34:56.1196,1900-12-25 -1900-05-05 12:34:56.1197,1900-12-25 -1900-05-05 12:34:56.1198,1900-12-25 -1900-05-05 12:34:56.1199,1900-12-25 -1900-05-05 12:34:56.12,1900-12-25 -1900-05-05 12:34:56.1201,1900-12-25 -1900-05-05 12:34:56.1202,1900-12-25 -1900-05-05 12:34:56.1203,1900-12-25 -1900-05-05 12:34:56.1204,1900-12-25 -1900-05-05 12:34:56.1205,1900-12-25 -1900-05-05 12:34:56.1206,1900-12-25 -1900-05-05 12:34:56.1207,1900-12-25 -1900-05-05 12:34:56.1208,1900-12-25 -1900-05-05 12:34:56.1209,1900-12-25 -1900-05-05 12:34:56.121,1900-12-25 -1900-05-05 12:34:56.1211,1900-12-25 -1900-05-05 12:34:56.1212,1900-12-25 -1900-05-05 12:34:56.1213,1900-12-25 -1900-05-05 12:34:56.1214,1900-12-25 -1900-05-05 12:34:56.1215,1900-12-25 -1900-05-05 12:34:56.1216,1900-12-25 -1900-05-05 12:34:56.1217,1900-12-25 -1900-05-05 12:34:56.1218,1900-12-25 -1900-05-05 12:34:56.1219,1900-12-25 -1900-05-05 12:34:56.122,1900-12-25 -1900-05-05 12:34:56.1221,1900-12-25 -1900-05-05 12:34:56.1222,1900-12-25 -1900-05-05 12:34:56.1223,1900-12-25 -1900-05-05 12:34:56.1224,1900-12-25 -1900-05-05 12:34:56.1225,1900-12-25 -1900-05-05 12:34:56.1226,1900-12-25 -1900-05-05 12:34:56.1227,1900-12-25 -1900-05-05 12:34:56.1228,1900-12-25 -1900-05-05 12:34:56.1229,1900-12-25 -1900-05-05 12:34:56.123,1900-12-25 -1900-05-05 12:34:56.1231,1900-12-25 -1900-05-05 12:34:56.1232,1900-12-25 -1900-05-05 12:34:56.1233,1900-12-25 -1900-05-05 12:34:56.1234,1900-12-25 -1900-05-05 12:34:56.1235,1900-12-25 -1900-05-05 12:34:56.1236,1900-12-25 -1900-05-05 12:34:56.1237,1900-12-25 -1900-05-05 12:34:56.1238,1900-12-25 -1900-05-05 12:34:56.1239,1900-12-25 -1900-05-05 12:34:56.124,1900-12-25 -1900-05-05 12:34:56.1241,1900-12-25 -1900-05-05 12:34:56.1242,1900-12-25 -1900-05-05 12:34:56.1243,1900-12-25 -1900-05-05 12:34:56.1244,1900-12-25 -1900-05-05 12:34:56.1245,1900-12-25 -1900-05-05 12:34:56.1246,1900-12-25 -1900-05-05 12:34:56.1247,1900-12-25 -1900-05-05 12:34:56.1248,1900-12-25 -1900-05-05 12:34:56.1249,1900-12-25 -1900-05-05 12:34:56.125,1900-12-25 -1900-05-05 12:34:56.1251,1900-12-25 -1900-05-05 12:34:56.1252,1900-12-25 -1900-05-05 12:34:56.1253,1900-12-25 -1900-05-05 12:34:56.1254,1900-12-25 -1900-05-05 12:34:56.1255,1900-12-25 -1900-05-05 12:34:56.1256,1900-12-25 -1900-05-05 12:34:56.1257,1900-12-25 -1900-05-05 12:34:56.1258,1900-12-25 -1900-05-05 12:34:56.1259,1900-12-25 -1900-05-05 12:34:56.126,1900-12-25 -1900-05-05 12:34:56.1261,1900-12-25 -1900-05-05 12:34:56.1262,1900-12-25 -1900-05-05 12:34:56.1263,1900-12-25 -1900-05-05 12:34:56.1264,1900-12-25 -1900-05-05 12:34:56.1265,1900-12-25 -1900-05-05 12:34:56.1266,1900-12-25 -1900-05-05 12:34:56.1267,1900-12-25 -1900-05-05 12:34:56.1268,1900-12-25 -1900-05-05 12:34:56.1269,1900-12-25 -1900-05-05 12:34:56.127,1900-12-25 -1900-05-05 12:34:56.1271,1900-12-25 -1900-05-05 12:34:56.1272,1900-12-25 -1900-05-05 12:34:56.1273,1900-12-25 -1900-05-05 12:34:56.1274,1900-12-25 -1900-05-05 12:34:56.1275,1900-12-25 -1900-05-05 12:34:56.1276,1900-12-25 -1900-05-05 12:34:56.1277,1900-12-25 -1900-05-05 12:34:56.1278,1900-12-25 -1900-05-05 12:34:56.1279,1900-12-25 -1900-05-05 12:34:56.128,1900-12-25 -1900-05-05 12:34:56.1281,1900-12-25 -1900-05-05 12:34:56.1282,1900-12-25 -1900-05-05 12:34:56.1283,1900-12-25 -1900-05-05 12:34:56.1284,1900-12-25 -1900-05-05 12:34:56.1285,1900-12-25 -1900-05-05 12:34:56.1286,1900-12-25 -1900-05-05 12:34:56.1287,1900-12-25 -1900-05-05 12:34:56.1288,1900-12-25 -1900-05-05 12:34:56.1289,1900-12-25 -1900-05-05 12:34:56.129,1900-12-25 -1900-05-05 12:34:56.1291,1900-12-25 -1900-05-05 12:34:56.1292,1900-12-25 -1900-05-05 12:34:56.1293,1900-12-25 -1900-05-05 12:34:56.1294,1900-12-25 -1900-05-05 12:34:56.1295,1900-12-25 -1900-05-05 12:34:56.1296,1900-12-25 -1900-05-05 12:34:56.1297,1900-12-25 -1900-05-05 12:34:56.1298,1900-12-25 -1900-05-05 12:34:56.1299,1900-12-25 -1900-05-05 12:34:56.13,1900-12-25 -1900-05-05 12:34:56.1301,1900-12-25 -1900-05-05 12:34:56.1302,1900-12-25 -1900-05-05 12:34:56.1303,1900-12-25 -1900-05-05 12:34:56.1304,1900-12-25 -1900-05-05 12:34:56.1305,1900-12-25 -1900-05-05 12:34:56.1306,1900-12-25 -1900-05-05 12:34:56.1307,1900-12-25 -1900-05-05 12:34:56.1308,1900-12-25 -1900-05-05 12:34:56.1309,1900-12-25 -1900-05-05 12:34:56.131,1900-12-25 -1900-05-05 12:34:56.1311,1900-12-25 -1900-05-05 12:34:56.1312,1900-12-25 -1900-05-05 12:34:56.1313,1900-12-25 -1900-05-05 12:34:56.1314,1900-12-25 -1900-05-05 12:34:56.1315,1900-12-25 -1900-05-05 12:34:56.1316,1900-12-25 -1900-05-05 12:34:56.1317,1900-12-25 -1900-05-05 12:34:56.1318,1900-12-25 -1900-05-05 12:34:56.1319,1900-12-25 -1900-05-05 12:34:56.132,1900-12-25 -1900-05-05 12:34:56.1321,1900-12-25 -1900-05-05 12:34:56.1322,1900-12-25 -1900-05-05 12:34:56.1323,1900-12-25 -1900-05-05 12:34:56.1324,1900-12-25 -1900-05-05 12:34:56.1325,1900-12-25 -1900-05-05 12:34:56.1326,1900-12-25 -1900-05-05 12:34:56.1327,1900-12-25 -1900-05-05 12:34:56.1328,1900-12-25 -1900-05-05 12:34:56.1329,1900-12-25 -1900-05-05 12:34:56.133,1900-12-25 -1900-05-05 12:34:56.1331,1900-12-25 -1900-05-05 12:34:56.1332,1900-12-25 -1900-05-05 12:34:56.1333,1900-12-25 -1900-05-05 12:34:56.1334,1900-12-25 -1900-05-05 12:34:56.1335,1900-12-25 -1900-05-05 12:34:56.1336,1900-12-25 -1900-05-05 12:34:56.1337,1900-12-25 -1900-05-05 12:34:56.1338,1900-12-25 -1900-05-05 12:34:56.1339,1900-12-25 -1900-05-05 12:34:56.134,1900-12-25 -1900-05-05 12:34:56.1341,1900-12-25 -1900-05-05 12:34:56.1342,1900-12-25 -1900-05-05 12:34:56.1343,1900-12-25 -1900-05-05 12:34:56.1344,1900-12-25 -1900-05-05 12:34:56.1345,1900-12-25 -1900-05-05 12:34:56.1346,1900-12-25 -1900-05-05 12:34:56.1347,1900-12-25 -1900-05-05 12:34:56.1348,1900-12-25 -1900-05-05 12:34:56.1349,1900-12-25 -1900-05-05 12:34:56.135,1900-12-25 -1900-05-05 12:34:56.1351,1900-12-25 -1900-05-05 12:34:56.1352,1900-12-25 -1900-05-05 12:34:56.1353,1900-12-25 -1900-05-05 12:34:56.1354,1900-12-25 -1900-05-05 12:34:56.1355,1900-12-25 -1900-05-05 12:34:56.1356,1900-12-25 -1900-05-05 12:34:56.1357,1900-12-25 -1900-05-05 12:34:56.1358,1900-12-25 -1900-05-05 12:34:56.1359,1900-12-25 -1900-05-05 12:34:56.136,1900-12-25 -1900-05-05 12:34:56.1361,1900-12-25 -1900-05-05 12:34:56.1362,1900-12-25 -1900-05-05 12:34:56.1363,1900-12-25 -1900-05-05 12:34:56.1364,1900-12-25 -1900-05-05 12:34:56.1365,1900-12-25 -1900-05-05 12:34:56.1366,1900-12-25 -1900-05-05 12:34:56.1367,1900-12-25 -1900-05-05 12:34:56.1368,1900-12-25 -1900-05-05 12:34:56.1369,1900-12-25 -1900-05-05 12:34:56.137,1900-12-25 -1900-05-05 12:34:56.1371,1900-12-25 -1900-05-05 12:34:56.1372,1900-12-25 -1900-05-05 12:34:56.1373,1900-12-25 -1900-05-05 12:34:56.1374,1900-12-25 -1900-05-05 12:34:56.1375,1900-12-25 -1900-05-05 12:34:56.1376,1900-12-25 -1900-05-05 12:34:56.1377,1900-12-25 -1900-05-05 12:34:56.1378,1900-12-25 -1900-05-05 12:34:56.1379,1900-12-25 -1900-05-05 12:34:56.138,1900-12-25 -1900-05-05 12:34:56.1381,1900-12-25 -1900-05-05 12:34:56.1382,1900-12-25 -1900-05-05 12:34:56.1383,1900-12-25 -1900-05-05 12:34:56.1384,1900-12-25 -1900-05-05 12:34:56.1385,1900-12-25 -1900-05-05 12:34:56.1386,1900-12-25 -1900-05-05 12:34:56.1387,1900-12-25 -1900-05-05 12:34:56.1388,1900-12-25 -1900-05-05 12:34:56.1389,1900-12-25 -1900-05-05 12:34:56.139,1900-12-25 -1900-05-05 12:34:56.1391,1900-12-25 -1900-05-05 12:34:56.1392,1900-12-25 -1900-05-05 12:34:56.1393,1900-12-25 -1900-05-05 12:34:56.1394,1900-12-25 -1900-05-05 12:34:56.1395,1900-12-25 -1900-05-05 12:34:56.1396,1900-12-25 -1900-05-05 12:34:56.1397,1900-12-25 -1900-05-05 12:34:56.1398,1900-12-25 -1900-05-05 12:34:56.1399,1900-12-25 -1900-05-05 12:34:56.14,1900-12-25 -1900-05-05 12:34:56.1401,1900-12-25 -1900-05-05 12:34:56.1402,1900-12-25 -1900-05-05 12:34:56.1403,1900-12-25 -1900-05-05 12:34:56.1404,1900-12-25 -1900-05-05 12:34:56.1405,1900-12-25 -1900-05-05 12:34:56.1406,1900-12-25 -1900-05-05 12:34:56.1407,1900-12-25 -1900-05-05 12:34:56.1408,1900-12-25 -1900-05-05 12:34:56.1409,1900-12-25 -1900-05-05 12:34:56.141,1900-12-25 -1900-05-05 12:34:56.1411,1900-12-25 -1900-05-05 12:34:56.1412,1900-12-25 -1900-05-05 12:34:56.1413,1900-12-25 -1900-05-05 12:34:56.1414,1900-12-25 -1900-05-05 12:34:56.1415,1900-12-25 -1900-05-05 12:34:56.1416,1900-12-25 -1900-05-05 12:34:56.1417,1900-12-25 -1900-05-05 12:34:56.1418,1900-12-25 -1900-05-05 12:34:56.1419,1900-12-25 -1900-05-05 12:34:56.142,1900-12-25 -1900-05-05 12:34:56.1421,1900-12-25 -1900-05-05 12:34:56.1422,1900-12-25 -1900-05-05 12:34:56.1423,1900-12-25 -1900-05-05 12:34:56.1424,1900-12-25 -1900-05-05 12:34:56.1425,1900-12-25 -1900-05-05 12:34:56.1426,1900-12-25 -1900-05-05 12:34:56.1427,1900-12-25 -1900-05-05 12:34:56.1428,1900-12-25 -1900-05-05 12:34:56.1429,1900-12-25 -1900-05-05 12:34:56.143,1900-12-25 -1900-05-05 12:34:56.1431,1900-12-25 -1900-05-05 12:34:56.1432,1900-12-25 -1900-05-05 12:34:56.1433,1900-12-25 -1900-05-05 12:34:56.1434,1900-12-25 -1900-05-05 12:34:56.1435,1900-12-25 -1900-05-05 12:34:56.1436,1900-12-25 -1900-05-05 12:34:56.1437,1900-12-25 -1900-05-05 12:34:56.1438,1900-12-25 -1900-05-05 12:34:56.1439,1900-12-25 -1900-05-05 12:34:56.144,1900-12-25 -1900-05-05 12:34:56.1441,1900-12-25 -1900-05-05 12:34:56.1442,1900-12-25 -1900-05-05 12:34:56.1443,1900-12-25 -1900-05-05 12:34:56.1444,1900-12-25 -1900-05-05 12:34:56.1445,1900-12-25 -1900-05-05 12:34:56.1446,1900-12-25 -1900-05-05 12:34:56.1447,1900-12-25 -1900-05-05 12:34:56.1448,1900-12-25 -1900-05-05 12:34:56.1449,1900-12-25 -1900-05-05 12:34:56.145,1900-12-25 -1900-05-05 12:34:56.1451,1900-12-25 -1900-05-05 12:34:56.1452,1900-12-25 -1900-05-05 12:34:56.1453,1900-12-25 -1900-05-05 12:34:56.1454,1900-12-25 -1900-05-05 12:34:56.1455,1900-12-25 -1900-05-05 12:34:56.1456,1900-12-25 -1900-05-05 12:34:56.1457,1900-12-25 -1900-05-05 12:34:56.1458,1900-12-25 -1900-05-05 12:34:56.1459,1900-12-25 -1900-05-05 12:34:56.146,1900-12-25 -1900-05-05 12:34:56.1461,1900-12-25 -1900-05-05 12:34:56.1462,1900-12-25 -1900-05-05 12:34:56.1463,1900-12-25 -1900-05-05 12:34:56.1464,1900-12-25 -1900-05-05 12:34:56.1465,1900-12-25 -1900-05-05 12:34:56.1466,1900-12-25 -1900-05-05 12:34:56.1467,1900-12-25 -1900-05-05 12:34:56.1468,1900-12-25 -1900-05-05 12:34:56.1469,1900-12-25 -1900-05-05 12:34:56.147,1900-12-25 -1900-05-05 12:34:56.1471,1900-12-25 -1900-05-05 12:34:56.1472,1900-12-25 -1900-05-05 12:34:56.1473,1900-12-25 -1900-05-05 12:34:56.1474,1900-12-25 -1900-05-05 12:34:56.1475,1900-12-25 -1900-05-05 12:34:56.1476,1900-12-25 -1900-05-05 12:34:56.1477,1900-12-25 -1900-05-05 12:34:56.1478,1900-12-25 -1900-05-05 12:34:56.1479,1900-12-25 -1900-05-05 12:34:56.148,1900-12-25 -1900-05-05 12:34:56.1481,1900-12-25 -1900-05-05 12:34:56.1482,1900-12-25 -1900-05-05 12:34:56.1483,1900-12-25 -1900-05-05 12:34:56.1484,1900-12-25 -1900-05-05 12:34:56.1485,1900-12-25 -1900-05-05 12:34:56.1486,1900-12-25 -1900-05-05 12:34:56.1487,1900-12-25 -1900-05-05 12:34:56.1488,1900-12-25 -1900-05-05 12:34:56.1489,1900-12-25 -1900-05-05 12:34:56.149,1900-12-25 -1900-05-05 12:34:56.1491,1900-12-25 -1900-05-05 12:34:56.1492,1900-12-25 -1900-05-05 12:34:56.1493,1900-12-25 -1900-05-05 12:34:56.1494,1900-12-25 -1900-05-05 12:34:56.1495,1900-12-25 -1900-05-05 12:34:56.1496,1900-12-25 -1900-05-05 12:34:56.1497,1900-12-25 -1900-05-05 12:34:56.1498,1900-12-25 -1900-05-05 12:34:56.1499,1900-12-25 -1900-05-05 12:34:56.15,1900-12-25 -1900-05-05 12:34:56.1501,1900-12-25 -1900-05-05 12:34:56.1502,1900-12-25 -1900-05-05 12:34:56.1503,1900-12-25 -1900-05-05 12:34:56.1504,1900-12-25 -1900-05-05 12:34:56.1505,1900-12-25 -1900-05-05 12:34:56.1506,1900-12-25 -1900-05-05 12:34:56.1507,1900-12-25 -1900-05-05 12:34:56.1508,1900-12-25 -1900-05-05 12:34:56.1509,1900-12-25 -1900-05-05 12:34:56.151,1900-12-25 -1900-05-05 12:34:56.1511,1900-12-25 -1900-05-05 12:34:56.1512,1900-12-25 -1900-05-05 12:34:56.1513,1900-12-25 -1900-05-05 12:34:56.1514,1900-12-25 -1900-05-05 12:34:56.1515,1900-12-25 -1900-05-05 12:34:56.1516,1900-12-25 -1900-05-05 12:34:56.1517,1900-12-25 -1900-05-05 12:34:56.1518,1900-12-25 -1900-05-05 12:34:56.1519,1900-12-25 -1900-05-05 12:34:56.152,1900-12-25 -1900-05-05 12:34:56.1521,1900-12-25 -1900-05-05 12:34:56.1522,1900-12-25 -1900-05-05 12:34:56.1523,1900-12-25 -1900-05-05 12:34:56.1524,1900-12-25 -1900-05-05 12:34:56.1525,1900-12-25 -1900-05-05 12:34:56.1526,1900-12-25 -1900-05-05 12:34:56.1527,1900-12-25 -1900-05-05 12:34:56.1528,1900-12-25 -1900-05-05 12:34:56.1529,1900-12-25 -1900-05-05 12:34:56.153,1900-12-25 -1900-05-05 12:34:56.1531,1900-12-25 -1900-05-05 12:34:56.1532,1900-12-25 -1900-05-05 12:34:56.1533,1900-12-25 -1900-05-05 12:34:56.1534,1900-12-25 -1900-05-05 12:34:56.1535,1900-12-25 -1900-05-05 12:34:56.1536,1900-12-25 -1900-05-05 12:34:56.1537,1900-12-25 -1900-05-05 12:34:56.1538,1900-12-25 -1900-05-05 12:34:56.1539,1900-12-25 -1900-05-05 12:34:56.154,1900-12-25 -1900-05-05 12:34:56.1541,1900-12-25 -1900-05-05 12:34:56.1542,1900-12-25 -1900-05-05 12:34:56.1543,1900-12-25 -1900-05-05 12:34:56.1544,1900-12-25 -1900-05-05 12:34:56.1545,1900-12-25 -1900-05-05 12:34:56.1546,1900-12-25 -1900-05-05 12:34:56.1547,1900-12-25 -1900-05-05 12:34:56.1548,1900-12-25 -1900-05-05 12:34:56.1549,1900-12-25 -1900-05-05 12:34:56.155,1900-12-25 -1900-05-05 12:34:56.1551,1900-12-25 -1900-05-05 12:34:56.1552,1900-12-25 -1900-05-05 12:34:56.1553,1900-12-25 -1900-05-05 12:34:56.1554,1900-12-25 -1900-05-05 12:34:56.1555,1900-12-25 -1900-05-05 12:34:56.1556,1900-12-25 -1900-05-05 12:34:56.1557,1900-12-25 -1900-05-05 12:34:56.1558,1900-12-25 -1900-05-05 12:34:56.1559,1900-12-25 -1900-05-05 12:34:56.156,1900-12-25 -1900-05-05 12:34:56.1561,1900-12-25 -1900-05-05 12:34:56.1562,1900-12-25 -1900-05-05 12:34:56.1563,1900-12-25 -1900-05-05 12:34:56.1564,1900-12-25 -1900-05-05 12:34:56.1565,1900-12-25 -1900-05-05 12:34:56.1566,1900-12-25 -1900-05-05 12:34:56.1567,1900-12-25 -1900-05-05 12:34:56.1568,1900-12-25 -1900-05-05 12:34:56.1569,1900-12-25 -1900-05-05 12:34:56.157,1900-12-25 -1900-05-05 12:34:56.1571,1900-12-25 -1900-05-05 12:34:56.1572,1900-12-25 -1900-05-05 12:34:56.1573,1900-12-25 -1900-05-05 12:34:56.1574,1900-12-25 -1900-05-05 12:34:56.1575,1900-12-25 -1900-05-05 12:34:56.1576,1900-12-25 -1900-05-05 12:34:56.1577,1900-12-25 -1900-05-05 12:34:56.1578,1900-12-25 -1900-05-05 12:34:56.1579,1900-12-25 -1900-05-05 12:34:56.158,1900-12-25 -1900-05-05 12:34:56.1581,1900-12-25 -1900-05-05 12:34:56.1582,1900-12-25 -1900-05-05 12:34:56.1583,1900-12-25 -1900-05-05 12:34:56.1584,1900-12-25 -1900-05-05 12:34:56.1585,1900-12-25 -1900-05-05 12:34:56.1586,1900-12-25 -1900-05-05 12:34:56.1587,1900-12-25 -1900-05-05 12:34:56.1588,1900-12-25 -1900-05-05 12:34:56.1589,1900-12-25 -1900-05-05 12:34:56.159,1900-12-25 -1900-05-05 12:34:56.1591,1900-12-25 -1900-05-05 12:34:56.1592,1900-12-25 -1900-05-05 12:34:56.1593,1900-12-25 -1900-05-05 12:34:56.1594,1900-12-25 -1900-05-05 12:34:56.1595,1900-12-25 -1900-05-05 12:34:56.1596,1900-12-25 -1900-05-05 12:34:56.1597,1900-12-25 -1900-05-05 12:34:56.1598,1900-12-25 -1900-05-05 12:34:56.1599,1900-12-25 -1900-05-05 12:34:56.16,1900-12-25 -1900-05-05 12:34:56.1601,1900-12-25 -1900-05-05 12:34:56.1602,1900-12-25 -1900-05-05 12:34:56.1603,1900-12-25 -1900-05-05 12:34:56.1604,1900-12-25 -1900-05-05 12:34:56.1605,1900-12-25 -1900-05-05 12:34:56.1606,1900-12-25 -1900-05-05 12:34:56.1607,1900-12-25 -1900-05-05 12:34:56.1608,1900-12-25 -1900-05-05 12:34:56.1609,1900-12-25 -1900-05-05 12:34:56.161,1900-12-25 -1900-05-05 12:34:56.1611,1900-12-25 -1900-05-05 12:34:56.1612,1900-12-25 -1900-05-05 12:34:56.1613,1900-12-25 -1900-05-05 12:34:56.1614,1900-12-25 -1900-05-05 12:34:56.1615,1900-12-25 -1900-05-05 12:34:56.1616,1900-12-25 -1900-05-05 12:34:56.1617,1900-12-25 -1900-05-05 12:34:56.1618,1900-12-25 -1900-05-05 12:34:56.1619,1900-12-25 -1900-05-05 12:34:56.162,1900-12-25 -1900-05-05 12:34:56.1621,1900-12-25 -1900-05-05 12:34:56.1622,1900-12-25 -1900-05-05 12:34:56.1623,1900-12-25 -1900-05-05 12:34:56.1624,1900-12-25 -1900-05-05 12:34:56.1625,1900-12-25 -1900-05-05 12:34:56.1626,1900-12-25 -1900-05-05 12:34:56.1627,1900-12-25 -1900-05-05 12:34:56.1628,1900-12-25 -1900-05-05 12:34:56.1629,1900-12-25 -1900-05-05 12:34:56.163,1900-12-25 -1900-05-05 12:34:56.1631,1900-12-25 -1900-05-05 12:34:56.1632,1900-12-25 -1900-05-05 12:34:56.1633,1900-12-25 -1900-05-05 12:34:56.1634,1900-12-25 -1900-05-05 12:34:56.1635,1900-12-25 -1900-05-05 12:34:56.1636,1900-12-25 -1900-05-05 12:34:56.1637,1900-12-25 -1900-05-05 12:34:56.1638,1900-12-25 -1900-05-05 12:34:56.1639,1900-12-25 -1900-05-05 12:34:56.164,1900-12-25 -1900-05-05 12:34:56.1641,1900-12-25 -1900-05-05 12:34:56.1642,1900-12-25 -1900-05-05 12:34:56.1643,1900-12-25 -1900-05-05 12:34:56.1644,1900-12-25 -1900-05-05 12:34:56.1645,1900-12-25 -1900-05-05 12:34:56.1646,1900-12-25 -1900-05-05 12:34:56.1647,1900-12-25 -1900-05-05 12:34:56.1648,1900-12-25 -1900-05-05 12:34:56.1649,1900-12-25 -1900-05-05 12:34:56.165,1900-12-25 -1900-05-05 12:34:56.1651,1900-12-25 -1900-05-05 12:34:56.1652,1900-12-25 -1900-05-05 12:34:56.1653,1900-12-25 -1900-05-05 12:34:56.1654,1900-12-25 -1900-05-05 12:34:56.1655,1900-12-25 -1900-05-05 12:34:56.1656,1900-12-25 -1900-05-05 12:34:56.1657,1900-12-25 -1900-05-05 12:34:56.1658,1900-12-25 -1900-05-05 12:34:56.1659,1900-12-25 -1900-05-05 12:34:56.166,1900-12-25 -1900-05-05 12:34:56.1661,1900-12-25 -1900-05-05 12:34:56.1662,1900-12-25 -1900-05-05 12:34:56.1663,1900-12-25 -1900-05-05 12:34:56.1664,1900-12-25 -1900-05-05 12:34:56.1665,1900-12-25 -1900-05-05 12:34:56.1666,1900-12-25 -1900-05-05 12:34:56.1667,1900-12-25 -1900-05-05 12:34:56.1668,1900-12-25 -1900-05-05 12:34:56.1669,1900-12-25 -1900-05-05 12:34:56.167,1900-12-25 -1900-05-05 12:34:56.1671,1900-12-25 -1900-05-05 12:34:56.1672,1900-12-25 -1900-05-05 12:34:56.1673,1900-12-25 -1900-05-05 12:34:56.1674,1900-12-25 -1900-05-05 12:34:56.1675,1900-12-25 -1900-05-05 12:34:56.1676,1900-12-25 -1900-05-05 12:34:56.1677,1900-12-25 -1900-05-05 12:34:56.1678,1900-12-25 -1900-05-05 12:34:56.1679,1900-12-25 -1900-05-05 12:34:56.168,1900-12-25 -1900-05-05 12:34:56.1681,1900-12-25 -1900-05-05 12:34:56.1682,1900-12-25 -1900-05-05 12:34:56.1683,1900-12-25 -1900-05-05 12:34:56.1684,1900-12-25 -1900-05-05 12:34:56.1685,1900-12-25 -1900-05-05 12:34:56.1686,1900-12-25 -1900-05-05 12:34:56.1687,1900-12-25 -1900-05-05 12:34:56.1688,1900-12-25 -1900-05-05 12:34:56.1689,1900-12-25 -1900-05-05 12:34:56.169,1900-12-25 -1900-05-05 12:34:56.1691,1900-12-25 -1900-05-05 12:34:56.1692,1900-12-25 -1900-05-05 12:34:56.1693,1900-12-25 -1900-05-05 12:34:56.1694,1900-12-25 -1900-05-05 12:34:56.1695,1900-12-25 -1900-05-05 12:34:56.1696,1900-12-25 -1900-05-05 12:34:56.1697,1900-12-25 -1900-05-05 12:34:56.1698,1900-12-25 -1900-05-05 12:34:56.1699,1900-12-25 -1900-05-05 12:34:56.17,1900-12-25 -1900-05-05 12:34:56.1701,1900-12-25 -1900-05-05 12:34:56.1702,1900-12-25 -1900-05-05 12:34:56.1703,1900-12-25 -1900-05-05 12:34:56.1704,1900-12-25 -1900-05-05 12:34:56.1705,1900-12-25 -1900-05-05 12:34:56.1706,1900-12-25 -1900-05-05 12:34:56.1707,1900-12-25 -1900-05-05 12:34:56.1708,1900-12-25 -1900-05-05 12:34:56.1709,1900-12-25 -1900-05-05 12:34:56.171,1900-12-25 -1900-05-05 12:34:56.1711,1900-12-25 -1900-05-05 12:34:56.1712,1900-12-25 -1900-05-05 12:34:56.1713,1900-12-25 -1900-05-05 12:34:56.1714,1900-12-25 -1900-05-05 12:34:56.1715,1900-12-25 -1900-05-05 12:34:56.1716,1900-12-25 -1900-05-05 12:34:56.1717,1900-12-25 -1900-05-05 12:34:56.1718,1900-12-25 -1900-05-05 12:34:56.1719,1900-12-25 -1900-05-05 12:34:56.172,1900-12-25 -1900-05-05 12:34:56.1721,1900-12-25 -1900-05-05 12:34:56.1722,1900-12-25 -1900-05-05 12:34:56.1723,1900-12-25 -1900-05-05 12:34:56.1724,1900-12-25 -1900-05-05 12:34:56.1725,1900-12-25 -1900-05-05 12:34:56.1726,1900-12-25 -1900-05-05 12:34:56.1727,1900-12-25 -1900-05-05 12:34:56.1728,1900-12-25 -1900-05-05 12:34:56.1729,1900-12-25 -1900-05-05 12:34:56.173,1900-12-25 -1900-05-05 12:34:56.1731,1900-12-25 -1900-05-05 12:34:56.1732,1900-12-25 -1900-05-05 12:34:56.1733,1900-12-25 -1900-05-05 12:34:56.1734,1900-12-25 -1900-05-05 12:34:56.1735,1900-12-25 -1900-05-05 12:34:56.1736,1900-12-25 -1900-05-05 12:34:56.1737,1900-12-25 -1900-05-05 12:34:56.1738,1900-12-25 -1900-05-05 12:34:56.1739,1900-12-25 -1900-05-05 12:34:56.174,1900-12-25 -1900-05-05 12:34:56.1741,1900-12-25 -1900-05-05 12:34:56.1742,1900-12-25 -1900-05-05 12:34:56.1743,1900-12-25 -1900-05-05 12:34:56.1744,1900-12-25 -1900-05-05 12:34:56.1745,1900-12-25 -1900-05-05 12:34:56.1746,1900-12-25 -1900-05-05 12:34:56.1747,1900-12-25 -1900-05-05 12:34:56.1748,1900-12-25 -1900-05-05 12:34:56.1749,1900-12-25 -1900-05-05 12:34:56.175,1900-12-25 -1900-05-05 12:34:56.1751,1900-12-25 -1900-05-05 12:34:56.1752,1900-12-25 -1900-05-05 12:34:56.1753,1900-12-25 -1900-05-05 12:34:56.1754,1900-12-25 -1900-05-05 12:34:56.1755,1900-12-25 -1900-05-05 12:34:56.1756,1900-12-25 -1900-05-05 12:34:56.1757,1900-12-25 -1900-05-05 12:34:56.1758,1900-12-25 -1900-05-05 12:34:56.1759,1900-12-25 -1900-05-05 12:34:56.176,1900-12-25 -1900-05-05 12:34:56.1761,1900-12-25 -1900-05-05 12:34:56.1762,1900-12-25 -1900-05-05 12:34:56.1763,1900-12-25 -1900-05-05 12:34:56.1764,1900-12-25 -1900-05-05 12:34:56.1765,1900-12-25 -1900-05-05 12:34:56.1766,1900-12-25 -1900-05-05 12:34:56.1767,1900-12-25 -1900-05-05 12:34:56.1768,1900-12-25 -1900-05-05 12:34:56.1769,1900-12-25 -1900-05-05 12:34:56.177,1900-12-25 -1900-05-05 12:34:56.1771,1900-12-25 -1900-05-05 12:34:56.1772,1900-12-25 -1900-05-05 12:34:56.1773,1900-12-25 -1900-05-05 12:34:56.1774,1900-12-25 -1900-05-05 12:34:56.1775,1900-12-25 -1900-05-05 12:34:56.1776,1900-12-25 -1900-05-05 12:34:56.1777,1900-12-25 -1900-05-05 12:34:56.1778,1900-12-25 -1900-05-05 12:34:56.1779,1900-12-25 -1900-05-05 12:34:56.178,1900-12-25 -1900-05-05 12:34:56.1781,1900-12-25 -1900-05-05 12:34:56.1782,1900-12-25 -1900-05-05 12:34:56.1783,1900-12-25 -1900-05-05 12:34:56.1784,1900-12-25 -1900-05-05 12:34:56.1785,1900-12-25 -1900-05-05 12:34:56.1786,1900-12-25 -1900-05-05 12:34:56.1787,1900-12-25 -1900-05-05 12:34:56.1788,1900-12-25 -1900-05-05 12:34:56.1789,1900-12-25 -1900-05-05 12:34:56.179,1900-12-25 -1900-05-05 12:34:56.1791,1900-12-25 -1900-05-05 12:34:56.1792,1900-12-25 -1900-05-05 12:34:56.1793,1900-12-25 -1900-05-05 12:34:56.1794,1900-12-25 -1900-05-05 12:34:56.1795,1900-12-25 -1900-05-05 12:34:56.1796,1900-12-25 -1900-05-05 12:34:56.1797,1900-12-25 -1900-05-05 12:34:56.1798,1900-12-25 -1900-05-05 12:34:56.1799,1900-12-25 -1900-05-05 12:34:56.18,1900-12-25 -1900-05-05 12:34:56.1801,1900-12-25 -1900-05-05 12:34:56.1802,1900-12-25 -1900-05-05 12:34:56.1803,1900-12-25 -1900-05-05 12:34:56.1804,1900-12-25 -1900-05-05 12:34:56.1805,1900-12-25 -1900-05-05 12:34:56.1806,1900-12-25 -1900-05-05 12:34:56.1807,1900-12-25 -1900-05-05 12:34:56.1808,1900-12-25 -1900-05-05 12:34:56.1809,1900-12-25 -1900-05-05 12:34:56.181,1900-12-25 -1900-05-05 12:34:56.1811,1900-12-25 -1900-05-05 12:34:56.1812,1900-12-25 -1900-05-05 12:34:56.1813,1900-12-25 -1900-05-05 12:34:56.1814,1900-12-25 -1900-05-05 12:34:56.1815,1900-12-25 -1900-05-05 12:34:56.1816,1900-12-25 -1900-05-05 12:34:56.1817,1900-12-25 -1900-05-05 12:34:56.1818,1900-12-25 -1900-05-05 12:34:56.1819,1900-12-25 -1900-05-05 12:34:56.182,1900-12-25 -1900-05-05 12:34:56.1821,1900-12-25 -1900-05-05 12:34:56.1822,1900-12-25 -1900-05-05 12:34:56.1823,1900-12-25 -1900-05-05 12:34:56.1824,1900-12-25 -1900-05-05 12:34:56.1825,1900-12-25 -1900-05-05 12:34:56.1826,1900-12-25 -1900-05-05 12:34:56.1827,1900-12-25 -1900-05-05 12:34:56.1828,1900-12-25 -1900-05-05 12:34:56.1829,1900-12-25 -1900-05-05 12:34:56.183,1900-12-25 -1900-05-05 12:34:56.1831,1900-12-25 -1900-05-05 12:34:56.1832,1900-12-25 -1900-05-05 12:34:56.1833,1900-12-25 -1900-05-05 12:34:56.1834,1900-12-25 -1900-05-05 12:34:56.1835,1900-12-25 -1900-05-05 12:34:56.1836,1900-12-25 -1900-05-05 12:34:56.1837,1900-12-25 -1900-05-05 12:34:56.1838,1900-12-25 -1900-05-05 12:34:56.1839,1900-12-25 -1900-05-05 12:34:56.184,1900-12-25 -1900-05-05 12:34:56.1841,1900-12-25 -1900-05-05 12:34:56.1842,1900-12-25 -1900-05-05 12:34:56.1843,1900-12-25 -1900-05-05 12:34:56.1844,1900-12-25 -1900-05-05 12:34:56.1845,1900-12-25 -1900-05-05 12:34:56.1846,1900-12-25 -1900-05-05 12:34:56.1847,1900-12-25 -1900-05-05 12:34:56.1848,1900-12-25 -1900-05-05 12:34:56.1849,1900-12-25 -1900-05-05 12:34:56.185,1900-12-25 -1900-05-05 12:34:56.1851,1900-12-25 -1900-05-05 12:34:56.1852,1900-12-25 -1900-05-05 12:34:56.1853,1900-12-25 -1900-05-05 12:34:56.1854,1900-12-25 -1900-05-05 12:34:56.1855,1900-12-25 -1900-05-05 12:34:56.1856,1900-12-25 -1900-05-05 12:34:56.1857,1900-12-25 -1900-05-05 12:34:56.1858,1900-12-25 -1900-05-05 12:34:56.1859,1900-12-25 -1900-05-05 12:34:56.186,1900-12-25 -1900-05-05 12:34:56.1861,1900-12-25 -1900-05-05 12:34:56.1862,1900-12-25 -1900-05-05 12:34:56.1863,1900-12-25 -1900-05-05 12:34:56.1864,1900-12-25 -1900-05-05 12:34:56.1865,1900-12-25 -1900-05-05 12:34:56.1866,1900-12-25 -1900-05-05 12:34:56.1867,1900-12-25 -1900-05-05 12:34:56.1868,1900-12-25 -1900-05-05 12:34:56.1869,1900-12-25 -1900-05-05 12:34:56.187,1900-12-25 -1900-05-05 12:34:56.1871,1900-12-25 -1900-05-05 12:34:56.1872,1900-12-25 -1900-05-05 12:34:56.1873,1900-12-25 -1900-05-05 12:34:56.1874,1900-12-25 -1900-05-05 12:34:56.1875,1900-12-25 -1900-05-05 12:34:56.1876,1900-12-25 -1900-05-05 12:34:56.1877,1900-12-25 -1900-05-05 12:34:56.1878,1900-12-25 -1900-05-05 12:34:56.1879,1900-12-25 -1900-05-05 12:34:56.188,1900-12-25 -1900-05-05 12:34:56.1881,1900-12-25 -1900-05-05 12:34:56.1882,1900-12-25 -1900-05-05 12:34:56.1883,1900-12-25 -1900-05-05 12:34:56.1884,1900-12-25 -1900-05-05 12:34:56.1885,1900-12-25 -1900-05-05 12:34:56.1886,1900-12-25 -1900-05-05 12:34:56.1887,1900-12-25 -1900-05-05 12:34:56.1888,1900-12-25 -1900-05-05 12:34:56.1889,1900-12-25 -1900-05-05 12:34:56.189,1900-12-25 -1900-05-05 12:34:56.1891,1900-12-25 -1900-05-05 12:34:56.1892,1900-12-25 -1900-05-05 12:34:56.1893,1900-12-25 -1900-05-05 12:34:56.1894,1900-12-25 -1900-05-05 12:34:56.1895,1900-12-25 -1900-05-05 12:34:56.1896,1900-12-25 -1900-05-05 12:34:56.1897,1900-12-25 -1900-05-05 12:34:56.1898,1900-12-25 -1900-05-05 12:34:56.1899,1900-12-25 -1900-05-05 12:34:56.19,1900-12-25 -1900-05-05 12:34:56.1901,1900-12-25 -1900-05-05 12:34:56.1902,1900-12-25 -1900-05-05 12:34:56.1903,1900-12-25 -1900-05-05 12:34:56.1904,1900-12-25 -1900-05-05 12:34:56.1905,1900-12-25 -1900-05-05 12:34:56.1906,1900-12-25 -1900-05-05 12:34:56.1907,1900-12-25 -1900-05-05 12:34:56.1908,1900-12-25 -1900-05-05 12:34:56.1909,1900-12-25 -1900-05-05 12:34:56.191,1900-12-25 -1900-05-05 12:34:56.1911,1900-12-25 -1900-05-05 12:34:56.1912,1900-12-25 -1900-05-05 12:34:56.1913,1900-12-25 -1900-05-05 12:34:56.1914,1900-12-25 -1900-05-05 12:34:56.1915,1900-12-25 -1900-05-05 12:34:56.1916,1900-12-25 -1900-05-05 12:34:56.1917,1900-12-25 -1900-05-05 12:34:56.1918,1900-12-25 -1900-05-05 12:34:56.1919,1900-12-25 -1900-05-05 12:34:56.192,1900-12-25 -1900-05-05 12:34:56.1921,1900-12-25 -1900-05-05 12:34:56.1922,1900-12-25 -1900-05-05 12:34:56.1923,1900-12-25 -1900-05-05 12:34:56.1924,1900-12-25 -1900-05-05 12:34:56.1925,1900-12-25 -1900-05-05 12:34:56.1926,1900-12-25 -1900-05-05 12:34:56.1927,1900-12-25 -1900-05-05 12:34:56.1928,1900-12-25 -1900-05-05 12:34:56.1929,1900-12-25 -1900-05-05 12:34:56.193,1900-12-25 -1900-05-05 12:34:56.1931,1900-12-25 -1900-05-05 12:34:56.1932,1900-12-25 -1900-05-05 12:34:56.1933,1900-12-25 -1900-05-05 12:34:56.1934,1900-12-25 -1900-05-05 12:34:56.1935,1900-12-25 -1900-05-05 12:34:56.1936,1900-12-25 -1900-05-05 12:34:56.1937,1900-12-25 -1900-05-05 12:34:56.1938,1900-12-25 -1900-05-05 12:34:56.1939,1900-12-25 -1900-05-05 12:34:56.194,1900-12-25 -1900-05-05 12:34:56.1941,1900-12-25 -1900-05-05 12:34:56.1942,1900-12-25 -1900-05-05 12:34:56.1943,1900-12-25 -1900-05-05 12:34:56.1944,1900-12-25 -1900-05-05 12:34:56.1945,1900-12-25 -1900-05-05 12:34:56.1946,1900-12-25 -1900-05-05 12:34:56.1947,1900-12-25 -1900-05-05 12:34:56.1948,1900-12-25 -1900-05-05 12:34:56.1949,1900-12-25 -1900-05-05 12:34:56.195,1900-12-25 -1900-05-05 12:34:56.1951,1900-12-25 -1900-05-05 12:34:56.1952,1900-12-25 -1900-05-05 12:34:56.1953,1900-12-25 -1900-05-05 12:34:56.1954,1900-12-25 -1900-05-05 12:34:56.1955,1900-12-25 -1900-05-05 12:34:56.1956,1900-12-25 -1900-05-05 12:34:56.1957,1900-12-25 -1900-05-05 12:34:56.1958,1900-12-25 -1900-05-05 12:34:56.1959,1900-12-25 -1900-05-05 12:34:56.196,1900-12-25 -1900-05-05 12:34:56.1961,1900-12-25 -1900-05-05 12:34:56.1962,1900-12-25 -1900-05-05 12:34:56.1963,1900-12-25 -1900-05-05 12:34:56.1964,1900-12-25 -1900-05-05 12:34:56.1965,1900-12-25 -1900-05-05 12:34:56.1966,1900-12-25 -1900-05-05 12:34:56.1967,1900-12-25 -1900-05-05 12:34:56.1968,1900-12-25 -1900-05-05 12:34:56.1969,1900-12-25 -1900-05-05 12:34:56.197,1900-12-25 -1900-05-05 12:34:56.1971,1900-12-25 -1900-05-05 12:34:56.1972,1900-12-25 -1900-05-05 12:34:56.1973,1900-12-25 -1900-05-05 12:34:56.1974,1900-12-25 -1900-05-05 12:34:56.1975,1900-12-25 -1900-05-05 12:34:56.1976,1900-12-25 -1900-05-05 12:34:56.1977,1900-12-25 -1900-05-05 12:34:56.1978,1900-12-25 -1900-05-05 12:34:56.1979,1900-12-25 -1900-05-05 12:34:56.198,1900-12-25 -1900-05-05 12:34:56.1981,1900-12-25 -1900-05-05 12:34:56.1982,1900-12-25 -1900-05-05 12:34:56.1983,1900-12-25 -1900-05-05 12:34:56.1984,1900-12-25 -1900-05-05 12:34:56.1985,1900-12-25 -1900-05-05 12:34:56.1986,1900-12-25 -1900-05-05 12:34:56.1987,1900-12-25 -1900-05-05 12:34:56.1988,1900-12-25 -1900-05-05 12:34:56.1989,1900-12-25 -1900-05-05 12:34:56.199,1900-12-25 -1900-05-05 12:34:56.1991,1900-12-25 -1900-05-05 12:34:56.1992,1900-12-25 -1900-05-05 12:34:56.1993,1900-12-25 -1900-05-05 12:34:56.1994,1900-12-25 -1900-05-05 12:34:56.1995,1900-12-25 -1900-05-05 12:34:56.1996,1900-12-25 -1900-05-05 12:34:56.1997,1900-12-25 -1900-05-05 12:34:56.1998,1900-12-25 -1900-05-05 12:34:56.1999,1900-12-25 -1901-05-05 12:34:56.1,1901-12-25 -1901-05-05 12:34:56.1001,1901-12-25 -1901-05-05 12:34:56.1002,1901-12-25 -1901-05-05 12:34:56.1003,1901-12-25 -1901-05-05 12:34:56.1004,1901-12-25 -1901-05-05 12:34:56.1005,1901-12-25 -1901-05-05 12:34:56.1006,1901-12-25 -1901-05-05 12:34:56.1007,1901-12-25 -1901-05-05 12:34:56.1008,1901-12-25 -1901-05-05 12:34:56.1009,1901-12-25 -1901-05-05 12:34:56.101,1901-12-25 -1901-05-05 12:34:56.1011,1901-12-25 -1901-05-05 12:34:56.1012,1901-12-25 -1901-05-05 12:34:56.1013,1901-12-25 -1901-05-05 12:34:56.1014,1901-12-25 -1901-05-05 12:34:56.1015,1901-12-25 -1901-05-05 12:34:56.1016,1901-12-25 -1901-05-05 12:34:56.1017,1901-12-25 -1901-05-05 12:34:56.1018,1901-12-25 -1901-05-05 12:34:56.1019,1901-12-25 -1901-05-05 12:34:56.102,1901-12-25 -1901-05-05 12:34:56.1021,1901-12-25 -1901-05-05 12:34:56.1022,1901-12-25 -1901-05-05 12:34:56.1023,1901-12-25 -1901-05-05 12:34:56.1024,1901-12-25 -1901-05-05 12:34:56.1025,1901-12-25 -1901-05-05 12:34:56.1026,1901-12-25 -1901-05-05 12:34:56.1027,1901-12-25 -1901-05-05 12:34:56.1028,1901-12-25 -1901-05-05 12:34:56.1029,1901-12-25 -1901-05-05 12:34:56.103,1901-12-25 -1901-05-05 12:34:56.1031,1901-12-25 -1901-05-05 12:34:56.1032,1901-12-25 -1901-05-05 12:34:56.1033,1901-12-25 -1901-05-05 12:34:56.1034,1901-12-25 -1901-05-05 12:34:56.1035,1901-12-25 -1901-05-05 12:34:56.1036,1901-12-25 -1901-05-05 12:34:56.1037,1901-12-25 -1901-05-05 12:34:56.1038,1901-12-25 -1901-05-05 12:34:56.1039,1901-12-25 -1901-05-05 12:34:56.104,1901-12-25 -1901-05-05 12:34:56.1041,1901-12-25 -1901-05-05 12:34:56.1042,1901-12-25 -1901-05-05 12:34:56.1043,1901-12-25 -1901-05-05 12:34:56.1044,1901-12-25 -1901-05-05 12:34:56.1045,1901-12-25 -1901-05-05 12:34:56.1046,1901-12-25 -1901-05-05 12:34:56.1047,1901-12-25 -1901-05-05 12:34:56.1048,1901-12-25 -1901-05-05 12:34:56.1049,1901-12-25 -1901-05-05 12:34:56.105,1901-12-25 -1901-05-05 12:34:56.1051,1901-12-25 -1901-05-05 12:34:56.1052,1901-12-25 -1901-05-05 12:34:56.1053,1901-12-25 -1901-05-05 12:34:56.1054,1901-12-25 -1901-05-05 12:34:56.1055,1901-12-25 -1901-05-05 12:34:56.1056,1901-12-25 -1901-05-05 12:34:56.1057,1901-12-25 -1901-05-05 12:34:56.1058,1901-12-25 -1901-05-05 12:34:56.1059,1901-12-25 -1901-05-05 12:34:56.106,1901-12-25 -1901-05-05 12:34:56.1061,1901-12-25 -1901-05-05 12:34:56.1062,1901-12-25 -1901-05-05 12:34:56.1063,1901-12-25 -1901-05-05 12:34:56.1064,1901-12-25 -1901-05-05 12:34:56.1065,1901-12-25 -1901-05-05 12:34:56.1066,1901-12-25 -1901-05-05 12:34:56.1067,1901-12-25 -1901-05-05 12:34:56.1068,1901-12-25 -1901-05-05 12:34:56.1069,1901-12-25 -1901-05-05 12:34:56.107,1901-12-25 -1901-05-05 12:34:56.1071,1901-12-25 -1901-05-05 12:34:56.1072,1901-12-25 -1901-05-05 12:34:56.1073,1901-12-25 -1901-05-05 12:34:56.1074,1901-12-25 -1901-05-05 12:34:56.1075,1901-12-25 -1901-05-05 12:34:56.1076,1901-12-25 -1901-05-05 12:34:56.1077,1901-12-25 -1901-05-05 12:34:56.1078,1901-12-25 -1901-05-05 12:34:56.1079,1901-12-25 -1901-05-05 12:34:56.108,1901-12-25 -1901-05-05 12:34:56.1081,1901-12-25 -1901-05-05 12:34:56.1082,1901-12-25 -1901-05-05 12:34:56.1083,1901-12-25 -1901-05-05 12:34:56.1084,1901-12-25 -1901-05-05 12:34:56.1085,1901-12-25 -1901-05-05 12:34:56.1086,1901-12-25 -1901-05-05 12:34:56.1087,1901-12-25 -1901-05-05 12:34:56.1088,1901-12-25 -1901-05-05 12:34:56.1089,1901-12-25 -1901-05-05 12:34:56.109,1901-12-25 -1901-05-05 12:34:56.1091,1901-12-25 -1901-05-05 12:34:56.1092,1901-12-25 -1901-05-05 12:34:56.1093,1901-12-25 -1901-05-05 12:34:56.1094,1901-12-25 -1901-05-05 12:34:56.1095,1901-12-25 -1901-05-05 12:34:56.1096,1901-12-25 -1901-05-05 12:34:56.1097,1901-12-25 -1901-05-05 12:34:56.1098,1901-12-25 -1901-05-05 12:34:56.1099,1901-12-25 -1901-05-05 12:34:56.11,1901-12-25 -1901-05-05 12:34:56.1101,1901-12-25 -1901-05-05 12:34:56.1102,1901-12-25 -1901-05-05 12:34:56.1103,1901-12-25 -1901-05-05 12:34:56.1104,1901-12-25 -1901-05-05 12:34:56.1105,1901-12-25 -1901-05-05 12:34:56.1106,1901-12-25 -1901-05-05 12:34:56.1107,1901-12-25 -1901-05-05 12:34:56.1108,1901-12-25 -1901-05-05 12:34:56.1109,1901-12-25 -1901-05-05 12:34:56.111,1901-12-25 -1901-05-05 12:34:56.1111,1901-12-25 -1901-05-05 12:34:56.1112,1901-12-25 -1901-05-05 12:34:56.1113,1901-12-25 -1901-05-05 12:34:56.1114,1901-12-25 -1901-05-05 12:34:56.1115,1901-12-25 -1901-05-05 12:34:56.1116,1901-12-25 -1901-05-05 12:34:56.1117,1901-12-25 -1901-05-05 12:34:56.1118,1901-12-25 -1901-05-05 12:34:56.1119,1901-12-25 -1901-05-05 12:34:56.112,1901-12-25 -1901-05-05 12:34:56.1121,1901-12-25 -1901-05-05 12:34:56.1122,1901-12-25 -1901-05-05 12:34:56.1123,1901-12-25 -1901-05-05 12:34:56.1124,1901-12-25 -1901-05-05 12:34:56.1125,1901-12-25 -1901-05-05 12:34:56.1126,1901-12-25 -1901-05-05 12:34:56.1127,1901-12-25 -1901-05-05 12:34:56.1128,1901-12-25 -1901-05-05 12:34:56.1129,1901-12-25 -1901-05-05 12:34:56.113,1901-12-25 -1901-05-05 12:34:56.1131,1901-12-25 -1901-05-05 12:34:56.1132,1901-12-25 -1901-05-05 12:34:56.1133,1901-12-25 -1901-05-05 12:34:56.1134,1901-12-25 -1901-05-05 12:34:56.1135,1901-12-25 -1901-05-05 12:34:56.1136,1901-12-25 -1901-05-05 12:34:56.1137,1901-12-25 -1901-05-05 12:34:56.1138,1901-12-25 -1901-05-05 12:34:56.1139,1901-12-25 -1901-05-05 12:34:56.114,1901-12-25 -1901-05-05 12:34:56.1141,1901-12-25 -1901-05-05 12:34:56.1142,1901-12-25 -1901-05-05 12:34:56.1143,1901-12-25 -1901-05-05 12:34:56.1144,1901-12-25 -1901-05-05 12:34:56.1145,1901-12-25 -1901-05-05 12:34:56.1146,1901-12-25 -1901-05-05 12:34:56.1147,1901-12-25 -1901-05-05 12:34:56.1148,1901-12-25 -1901-05-05 12:34:56.1149,1901-12-25 -1901-05-05 12:34:56.115,1901-12-25 -1901-05-05 12:34:56.1151,1901-12-25 -1901-05-05 12:34:56.1152,1901-12-25 -1901-05-05 12:34:56.1153,1901-12-25 -1901-05-05 12:34:56.1154,1901-12-25 -1901-05-05 12:34:56.1155,1901-12-25 -1901-05-05 12:34:56.1156,1901-12-25 -1901-05-05 12:34:56.1157,1901-12-25 -1901-05-05 12:34:56.1158,1901-12-25 -1901-05-05 12:34:56.1159,1901-12-25 -1901-05-05 12:34:56.116,1901-12-25 -1901-05-05 12:34:56.1161,1901-12-25 -1901-05-05 12:34:56.1162,1901-12-25 -1901-05-05 12:34:56.1163,1901-12-25 -1901-05-05 12:34:56.1164,1901-12-25 -1901-05-05 12:34:56.1165,1901-12-25 -1901-05-05 12:34:56.1166,1901-12-25 -1901-05-05 12:34:56.1167,1901-12-25 -1901-05-05 12:34:56.1168,1901-12-25 -1901-05-05 12:34:56.1169,1901-12-25 -1901-05-05 12:34:56.117,1901-12-25 -1901-05-05 12:34:56.1171,1901-12-25 -1901-05-05 12:34:56.1172,1901-12-25 -1901-05-05 12:34:56.1173,1901-12-25 -1901-05-05 12:34:56.1174,1901-12-25 -1901-05-05 12:34:56.1175,1901-12-25 -1901-05-05 12:34:56.1176,1901-12-25 -1901-05-05 12:34:56.1177,1901-12-25 -1901-05-05 12:34:56.1178,1901-12-25 -1901-05-05 12:34:56.1179,1901-12-25 -1901-05-05 12:34:56.118,1901-12-25 -1901-05-05 12:34:56.1181,1901-12-25 -1901-05-05 12:34:56.1182,1901-12-25 -1901-05-05 12:34:56.1183,1901-12-25 -1901-05-05 12:34:56.1184,1901-12-25 -1901-05-05 12:34:56.1185,1901-12-25 -1901-05-05 12:34:56.1186,1901-12-25 -1901-05-05 12:34:56.1187,1901-12-25 -1901-05-05 12:34:56.1188,1901-12-25 -1901-05-05 12:34:56.1189,1901-12-25 -1901-05-05 12:34:56.119,1901-12-25 -1901-05-05 12:34:56.1191,1901-12-25 -1901-05-05 12:34:56.1192,1901-12-25 -1901-05-05 12:34:56.1193,1901-12-25 -1901-05-05 12:34:56.1194,1901-12-25 -1901-05-05 12:34:56.1195,1901-12-25 -1901-05-05 12:34:56.1196,1901-12-25 -1901-05-05 12:34:56.1197,1901-12-25 -1901-05-05 12:34:56.1198,1901-12-25 -1901-05-05 12:34:56.1199,1901-12-25 -1901-05-05 12:34:56.12,1901-12-25 -1901-05-05 12:34:56.1201,1901-12-25 -1901-05-05 12:34:56.1202,1901-12-25 -1901-05-05 12:34:56.1203,1901-12-25 -1901-05-05 12:34:56.1204,1901-12-25 -1901-05-05 12:34:56.1205,1901-12-25 -1901-05-05 12:34:56.1206,1901-12-25 -1901-05-05 12:34:56.1207,1901-12-25 -1901-05-05 12:34:56.1208,1901-12-25 -1901-05-05 12:34:56.1209,1901-12-25 -1901-05-05 12:34:56.121,1901-12-25 -1901-05-05 12:34:56.1211,1901-12-25 -1901-05-05 12:34:56.1212,1901-12-25 -1901-05-05 12:34:56.1213,1901-12-25 -1901-05-05 12:34:56.1214,1901-12-25 -1901-05-05 12:34:56.1215,1901-12-25 -1901-05-05 12:34:56.1216,1901-12-25 -1901-05-05 12:34:56.1217,1901-12-25 -1901-05-05 12:34:56.1218,1901-12-25 -1901-05-05 12:34:56.1219,1901-12-25 -1901-05-05 12:34:56.122,1901-12-25 -1901-05-05 12:34:56.1221,1901-12-25 -1901-05-05 12:34:56.1222,1901-12-25 -1901-05-05 12:34:56.1223,1901-12-25 -1901-05-05 12:34:56.1224,1901-12-25 -1901-05-05 12:34:56.1225,1901-12-25 -1901-05-05 12:34:56.1226,1901-12-25 -1901-05-05 12:34:56.1227,1901-12-25 -1901-05-05 12:34:56.1228,1901-12-25 -1901-05-05 12:34:56.1229,1901-12-25 -1901-05-05 12:34:56.123,1901-12-25 -1901-05-05 12:34:56.1231,1901-12-25 -1901-05-05 12:34:56.1232,1901-12-25 -1901-05-05 12:34:56.1233,1901-12-25 -1901-05-05 12:34:56.1234,1901-12-25 -1901-05-05 12:34:56.1235,1901-12-25 -1901-05-05 12:34:56.1236,1901-12-25 -1901-05-05 12:34:56.1237,1901-12-25 -1901-05-05 12:34:56.1238,1901-12-25 -1901-05-05 12:34:56.1239,1901-12-25 -1901-05-05 12:34:56.124,1901-12-25 -1901-05-05 12:34:56.1241,1901-12-25 -1901-05-05 12:34:56.1242,1901-12-25 -1901-05-05 12:34:56.1243,1901-12-25 -1901-05-05 12:34:56.1244,1901-12-25 -1901-05-05 12:34:56.1245,1901-12-25 -1901-05-05 12:34:56.1246,1901-12-25 -1901-05-05 12:34:56.1247,1901-12-25 -1901-05-05 12:34:56.1248,1901-12-25 -1901-05-05 12:34:56.1249,1901-12-25 -1901-05-05 12:34:56.125,1901-12-25 -1901-05-05 12:34:56.1251,1901-12-25 -1901-05-05 12:34:56.1252,1901-12-25 -1901-05-05 12:34:56.1253,1901-12-25 -1901-05-05 12:34:56.1254,1901-12-25 -1901-05-05 12:34:56.1255,1901-12-25 -1901-05-05 12:34:56.1256,1901-12-25 -1901-05-05 12:34:56.1257,1901-12-25 -1901-05-05 12:34:56.1258,1901-12-25 -1901-05-05 12:34:56.1259,1901-12-25 -1901-05-05 12:34:56.126,1901-12-25 -1901-05-05 12:34:56.1261,1901-12-25 -1901-05-05 12:34:56.1262,1901-12-25 -1901-05-05 12:34:56.1263,1901-12-25 -1901-05-05 12:34:56.1264,1901-12-25 -1901-05-05 12:34:56.1265,1901-12-25 -1901-05-05 12:34:56.1266,1901-12-25 -1901-05-05 12:34:56.1267,1901-12-25 -1901-05-05 12:34:56.1268,1901-12-25 -1901-05-05 12:34:56.1269,1901-12-25 -1901-05-05 12:34:56.127,1901-12-25 -1901-05-05 12:34:56.1271,1901-12-25 -1901-05-05 12:34:56.1272,1901-12-25 -1901-05-05 12:34:56.1273,1901-12-25 -1901-05-05 12:34:56.1274,1901-12-25 -1901-05-05 12:34:56.1275,1901-12-25 -1901-05-05 12:34:56.1276,1901-12-25 -1901-05-05 12:34:56.1277,1901-12-25 -1901-05-05 12:34:56.1278,1901-12-25 -1901-05-05 12:34:56.1279,1901-12-25 -1901-05-05 12:34:56.128,1901-12-25 -1901-05-05 12:34:56.1281,1901-12-25 -1901-05-05 12:34:56.1282,1901-12-25 -1901-05-05 12:34:56.1283,1901-12-25 -1901-05-05 12:34:56.1284,1901-12-25 -1901-05-05 12:34:56.1285,1901-12-25 -1901-05-05 12:34:56.1286,1901-12-25 -1901-05-05 12:34:56.1287,1901-12-25 -1901-05-05 12:34:56.1288,1901-12-25 -1901-05-05 12:34:56.1289,1901-12-25 -1901-05-05 12:34:56.129,1901-12-25 -1901-05-05 12:34:56.1291,1901-12-25 -1901-05-05 12:34:56.1292,1901-12-25 -1901-05-05 12:34:56.1293,1901-12-25 -1901-05-05 12:34:56.1294,1901-12-25 -1901-05-05 12:34:56.1295,1901-12-25 -1901-05-05 12:34:56.1296,1901-12-25 -1901-05-05 12:34:56.1297,1901-12-25 -1901-05-05 12:34:56.1298,1901-12-25 -1901-05-05 12:34:56.1299,1901-12-25 -1901-05-05 12:34:56.13,1901-12-25 -1901-05-05 12:34:56.1301,1901-12-25 -1901-05-05 12:34:56.1302,1901-12-25 -1901-05-05 12:34:56.1303,1901-12-25 -1901-05-05 12:34:56.1304,1901-12-25 -1901-05-05 12:34:56.1305,1901-12-25 -1901-05-05 12:34:56.1306,1901-12-25 -1901-05-05 12:34:56.1307,1901-12-25 -1901-05-05 12:34:56.1308,1901-12-25 -1901-05-05 12:34:56.1309,1901-12-25 -1901-05-05 12:34:56.131,1901-12-25 -1901-05-05 12:34:56.1311,1901-12-25 -1901-05-05 12:34:56.1312,1901-12-25 -1901-05-05 12:34:56.1313,1901-12-25 -1901-05-05 12:34:56.1314,1901-12-25 -1901-05-05 12:34:56.1315,1901-12-25 -1901-05-05 12:34:56.1316,1901-12-25 -1901-05-05 12:34:56.1317,1901-12-25 -1901-05-05 12:34:56.1318,1901-12-25 -1901-05-05 12:34:56.1319,1901-12-25 -1901-05-05 12:34:56.132,1901-12-25 -1901-05-05 12:34:56.1321,1901-12-25 -1901-05-05 12:34:56.1322,1901-12-25 -1901-05-05 12:34:56.1323,1901-12-25 -1901-05-05 12:34:56.1324,1901-12-25 -1901-05-05 12:34:56.1325,1901-12-25 -1901-05-05 12:34:56.1326,1901-12-25 -1901-05-05 12:34:56.1327,1901-12-25 -1901-05-05 12:34:56.1328,1901-12-25 -1901-05-05 12:34:56.1329,1901-12-25 -1901-05-05 12:34:56.133,1901-12-25 -1901-05-05 12:34:56.1331,1901-12-25 -1901-05-05 12:34:56.1332,1901-12-25 -1901-05-05 12:34:56.1333,1901-12-25 -1901-05-05 12:34:56.1334,1901-12-25 -1901-05-05 12:34:56.1335,1901-12-25 -1901-05-05 12:34:56.1336,1901-12-25 -1901-05-05 12:34:56.1337,1901-12-25 -1901-05-05 12:34:56.1338,1901-12-25 -1901-05-05 12:34:56.1339,1901-12-25 -1901-05-05 12:34:56.134,1901-12-25 -1901-05-05 12:34:56.1341,1901-12-25 -1901-05-05 12:34:56.1342,1901-12-25 -1901-05-05 12:34:56.1343,1901-12-25 -1901-05-05 12:34:56.1344,1901-12-25 -1901-05-05 12:34:56.1345,1901-12-25 -1901-05-05 12:34:56.1346,1901-12-25 -1901-05-05 12:34:56.1347,1901-12-25 -1901-05-05 12:34:56.1348,1901-12-25 -1901-05-05 12:34:56.1349,1901-12-25 -1901-05-05 12:34:56.135,1901-12-25 -1901-05-05 12:34:56.1351,1901-12-25 -1901-05-05 12:34:56.1352,1901-12-25 -1901-05-05 12:34:56.1353,1901-12-25 -1901-05-05 12:34:56.1354,1901-12-25 -1901-05-05 12:34:56.1355,1901-12-25 -1901-05-05 12:34:56.1356,1901-12-25 -1901-05-05 12:34:56.1357,1901-12-25 -1901-05-05 12:34:56.1358,1901-12-25 -1901-05-05 12:34:56.1359,1901-12-25 -1901-05-05 12:34:56.136,1901-12-25 -1901-05-05 12:34:56.1361,1901-12-25 -1901-05-05 12:34:56.1362,1901-12-25 -1901-05-05 12:34:56.1363,1901-12-25 -1901-05-05 12:34:56.1364,1901-12-25 -1901-05-05 12:34:56.1365,1901-12-25 -1901-05-05 12:34:56.1366,1901-12-25 -1901-05-05 12:34:56.1367,1901-12-25 -1901-05-05 12:34:56.1368,1901-12-25 -1901-05-05 12:34:56.1369,1901-12-25 -1901-05-05 12:34:56.137,1901-12-25 -1901-05-05 12:34:56.1371,1901-12-25 -1901-05-05 12:34:56.1372,1901-12-25 -1901-05-05 12:34:56.1373,1901-12-25 -1901-05-05 12:34:56.1374,1901-12-25 -1901-05-05 12:34:56.1375,1901-12-25 -1901-05-05 12:34:56.1376,1901-12-25 -1901-05-05 12:34:56.1377,1901-12-25 -1901-05-05 12:34:56.1378,1901-12-25 -1901-05-05 12:34:56.1379,1901-12-25 -1901-05-05 12:34:56.138,1901-12-25 -1901-05-05 12:34:56.1381,1901-12-25 -1901-05-05 12:34:56.1382,1901-12-25 -1901-05-05 12:34:56.1383,1901-12-25 -1901-05-05 12:34:56.1384,1901-12-25 -1901-05-05 12:34:56.1385,1901-12-25 -1901-05-05 12:34:56.1386,1901-12-25 -1901-05-05 12:34:56.1387,1901-12-25 -1901-05-05 12:34:56.1388,1901-12-25 -1901-05-05 12:34:56.1389,1901-12-25 -1901-05-05 12:34:56.139,1901-12-25 -1901-05-05 12:34:56.1391,1901-12-25 -1901-05-05 12:34:56.1392,1901-12-25 -1901-05-05 12:34:56.1393,1901-12-25 -1901-05-05 12:34:56.1394,1901-12-25 -1901-05-05 12:34:56.1395,1901-12-25 -1901-05-05 12:34:56.1396,1901-12-25 -1901-05-05 12:34:56.1397,1901-12-25 -1901-05-05 12:34:56.1398,1901-12-25 -1901-05-05 12:34:56.1399,1901-12-25 -1901-05-05 12:34:56.14,1901-12-25 -1901-05-05 12:34:56.1401,1901-12-25 -1901-05-05 12:34:56.1402,1901-12-25 -1901-05-05 12:34:56.1403,1901-12-25 -1901-05-05 12:34:56.1404,1901-12-25 -1901-05-05 12:34:56.1405,1901-12-25 -1901-05-05 12:34:56.1406,1901-12-25 -1901-05-05 12:34:56.1407,1901-12-25 -1901-05-05 12:34:56.1408,1901-12-25 -1901-05-05 12:34:56.1409,1901-12-25 -1901-05-05 12:34:56.141,1901-12-25 -1901-05-05 12:34:56.1411,1901-12-25 -1901-05-05 12:34:56.1412,1901-12-25 -1901-05-05 12:34:56.1413,1901-12-25 -1901-05-05 12:34:56.1414,1901-12-25 -1901-05-05 12:34:56.1415,1901-12-25 -1901-05-05 12:34:56.1416,1901-12-25 -1901-05-05 12:34:56.1417,1901-12-25 -1901-05-05 12:34:56.1418,1901-12-25 -1901-05-05 12:34:56.1419,1901-12-25 -1901-05-05 12:34:56.142,1901-12-25 -1901-05-05 12:34:56.1421,1901-12-25 -1901-05-05 12:34:56.1422,1901-12-25 -1901-05-05 12:34:56.1423,1901-12-25 -1901-05-05 12:34:56.1424,1901-12-25 -1901-05-05 12:34:56.1425,1901-12-25 -1901-05-05 12:34:56.1426,1901-12-25 -1901-05-05 12:34:56.1427,1901-12-25 -1901-05-05 12:34:56.1428,1901-12-25 -1901-05-05 12:34:56.1429,1901-12-25 -1901-05-05 12:34:56.143,1901-12-25 -1901-05-05 12:34:56.1431,1901-12-25 -1901-05-05 12:34:56.1432,1901-12-25 -1901-05-05 12:34:56.1433,1901-12-25 -1901-05-05 12:34:56.1434,1901-12-25 -1901-05-05 12:34:56.1435,1901-12-25 -1901-05-05 12:34:56.1436,1901-12-25 -1901-05-05 12:34:56.1437,1901-12-25 -1901-05-05 12:34:56.1438,1901-12-25 -1901-05-05 12:34:56.1439,1901-12-25 -1901-05-05 12:34:56.144,1901-12-25 -1901-05-05 12:34:56.1441,1901-12-25 -1901-05-05 12:34:56.1442,1901-12-25 -1901-05-05 12:34:56.1443,1901-12-25 -1901-05-05 12:34:56.1444,1901-12-25 -1901-05-05 12:34:56.1445,1901-12-25 -1901-05-05 12:34:56.1446,1901-12-25 -1901-05-05 12:34:56.1447,1901-12-25 -1901-05-05 12:34:56.1448,1901-12-25 -1901-05-05 12:34:56.1449,1901-12-25 -1901-05-05 12:34:56.145,1901-12-25 -1901-05-05 12:34:56.1451,1901-12-25 -1901-05-05 12:34:56.1452,1901-12-25 -1901-05-05 12:34:56.1453,1901-12-25 -1901-05-05 12:34:56.1454,1901-12-25 -1901-05-05 12:34:56.1455,1901-12-25 -1901-05-05 12:34:56.1456,1901-12-25 -1901-05-05 12:34:56.1457,1901-12-25 -1901-05-05 12:34:56.1458,1901-12-25 -1901-05-05 12:34:56.1459,1901-12-25 -1901-05-05 12:34:56.146,1901-12-25 -1901-05-05 12:34:56.1461,1901-12-25 -1901-05-05 12:34:56.1462,1901-12-25 -1901-05-05 12:34:56.1463,1901-12-25 -1901-05-05 12:34:56.1464,1901-12-25 -1901-05-05 12:34:56.1465,1901-12-25 -1901-05-05 12:34:56.1466,1901-12-25 -1901-05-05 12:34:56.1467,1901-12-25 -1901-05-05 12:34:56.1468,1901-12-25 -1901-05-05 12:34:56.1469,1901-12-25 -1901-05-05 12:34:56.147,1901-12-25 -1901-05-05 12:34:56.1471,1901-12-25 -1901-05-05 12:34:56.1472,1901-12-25 -1901-05-05 12:34:56.1473,1901-12-25 -1901-05-05 12:34:56.1474,1901-12-25 -1901-05-05 12:34:56.1475,1901-12-25 -1901-05-05 12:34:56.1476,1901-12-25 -1901-05-05 12:34:56.1477,1901-12-25 -1901-05-05 12:34:56.1478,1901-12-25 -1901-05-05 12:34:56.1479,1901-12-25 -1901-05-05 12:34:56.148,1901-12-25 -1901-05-05 12:34:56.1481,1901-12-25 -1901-05-05 12:34:56.1482,1901-12-25 -1901-05-05 12:34:56.1483,1901-12-25 -1901-05-05 12:34:56.1484,1901-12-25 -1901-05-05 12:34:56.1485,1901-12-25 -1901-05-05 12:34:56.1486,1901-12-25 -1901-05-05 12:34:56.1487,1901-12-25 -1901-05-05 12:34:56.1488,1901-12-25 -1901-05-05 12:34:56.1489,1901-12-25 -1901-05-05 12:34:56.149,1901-12-25 -1901-05-05 12:34:56.1491,1901-12-25 -1901-05-05 12:34:56.1492,1901-12-25 -1901-05-05 12:34:56.1493,1901-12-25 -1901-05-05 12:34:56.1494,1901-12-25 -1901-05-05 12:34:56.1495,1901-12-25 -1901-05-05 12:34:56.1496,1901-12-25 -1901-05-05 12:34:56.1497,1901-12-25 -1901-05-05 12:34:56.1498,1901-12-25 -1901-05-05 12:34:56.1499,1901-12-25 -1901-05-05 12:34:56.15,1901-12-25 -1901-05-05 12:34:56.1501,1901-12-25 -1901-05-05 12:34:56.1502,1901-12-25 -1901-05-05 12:34:56.1503,1901-12-25 -1901-05-05 12:34:56.1504,1901-12-25 -1901-05-05 12:34:56.1505,1901-12-25 -1901-05-05 12:34:56.1506,1901-12-25 -1901-05-05 12:34:56.1507,1901-12-25 -1901-05-05 12:34:56.1508,1901-12-25 -1901-05-05 12:34:56.1509,1901-12-25 -1901-05-05 12:34:56.151,1901-12-25 -1901-05-05 12:34:56.1511,1901-12-25 -1901-05-05 12:34:56.1512,1901-12-25 -1901-05-05 12:34:56.1513,1901-12-25 -1901-05-05 12:34:56.1514,1901-12-25 -1901-05-05 12:34:56.1515,1901-12-25 -1901-05-05 12:34:56.1516,1901-12-25 -1901-05-05 12:34:56.1517,1901-12-25 -1901-05-05 12:34:56.1518,1901-12-25 -1901-05-05 12:34:56.1519,1901-12-25 -1901-05-05 12:34:56.152,1901-12-25 -1901-05-05 12:34:56.1521,1901-12-25 -1901-05-05 12:34:56.1522,1901-12-25 -1901-05-05 12:34:56.1523,1901-12-25 -1901-05-05 12:34:56.1524,1901-12-25 -1901-05-05 12:34:56.1525,1901-12-25 -1901-05-05 12:34:56.1526,1901-12-25 -1901-05-05 12:34:56.1527,1901-12-25 -1901-05-05 12:34:56.1528,1901-12-25 -1901-05-05 12:34:56.1529,1901-12-25 -1901-05-05 12:34:56.153,1901-12-25 -1901-05-05 12:34:56.1531,1901-12-25 -1901-05-05 12:34:56.1532,1901-12-25 -1901-05-05 12:34:56.1533,1901-12-25 -1901-05-05 12:34:56.1534,1901-12-25 -1901-05-05 12:34:56.1535,1901-12-25 -1901-05-05 12:34:56.1536,1901-12-25 -1901-05-05 12:34:56.1537,1901-12-25 -1901-05-05 12:34:56.1538,1901-12-25 -1901-05-05 12:34:56.1539,1901-12-25 -1901-05-05 12:34:56.154,1901-12-25 -1901-05-05 12:34:56.1541,1901-12-25 -1901-05-05 12:34:56.1542,1901-12-25 -1901-05-05 12:34:56.1543,1901-12-25 -1901-05-05 12:34:56.1544,1901-12-25 -1901-05-05 12:34:56.1545,1901-12-25 -1901-05-05 12:34:56.1546,1901-12-25 -1901-05-05 12:34:56.1547,1901-12-25 -1901-05-05 12:34:56.1548,1901-12-25 -1901-05-05 12:34:56.1549,1901-12-25 -1901-05-05 12:34:56.155,1901-12-25 -1901-05-05 12:34:56.1551,1901-12-25 -1901-05-05 12:34:56.1552,1901-12-25 -1901-05-05 12:34:56.1553,1901-12-25 -1901-05-05 12:34:56.1554,1901-12-25 -1901-05-05 12:34:56.1555,1901-12-25 -1901-05-05 12:34:56.1556,1901-12-25 -1901-05-05 12:34:56.1557,1901-12-25 -1901-05-05 12:34:56.1558,1901-12-25 -1901-05-05 12:34:56.1559,1901-12-25 -1901-05-05 12:34:56.156,1901-12-25 -1901-05-05 12:34:56.1561,1901-12-25 -1901-05-05 12:34:56.1562,1901-12-25 -1901-05-05 12:34:56.1563,1901-12-25 -1901-05-05 12:34:56.1564,1901-12-25 -1901-05-05 12:34:56.1565,1901-12-25 -1901-05-05 12:34:56.1566,1901-12-25 -1901-05-05 12:34:56.1567,1901-12-25 -1901-05-05 12:34:56.1568,1901-12-25 -1901-05-05 12:34:56.1569,1901-12-25 -1901-05-05 12:34:56.157,1901-12-25 -1901-05-05 12:34:56.1571,1901-12-25 -1901-05-05 12:34:56.1572,1901-12-25 -1901-05-05 12:34:56.1573,1901-12-25 -1901-05-05 12:34:56.1574,1901-12-25 -1901-05-05 12:34:56.1575,1901-12-25 -1901-05-05 12:34:56.1576,1901-12-25 -1901-05-05 12:34:56.1577,1901-12-25 -1901-05-05 12:34:56.1578,1901-12-25 -1901-05-05 12:34:56.1579,1901-12-25 -1901-05-05 12:34:56.158,1901-12-25 -1901-05-05 12:34:56.1581,1901-12-25 -1901-05-05 12:34:56.1582,1901-12-25 -1901-05-05 12:34:56.1583,1901-12-25 -1901-05-05 12:34:56.1584,1901-12-25 -1901-05-05 12:34:56.1585,1901-12-25 -1901-05-05 12:34:56.1586,1901-12-25 -1901-05-05 12:34:56.1587,1901-12-25 -1901-05-05 12:34:56.1588,1901-12-25 -1901-05-05 12:34:56.1589,1901-12-25 -1901-05-05 12:34:56.159,1901-12-25 -1901-05-05 12:34:56.1591,1901-12-25 -1901-05-05 12:34:56.1592,1901-12-25 -1901-05-05 12:34:56.1593,1901-12-25 -1901-05-05 12:34:56.1594,1901-12-25 -1901-05-05 12:34:56.1595,1901-12-25 -1901-05-05 12:34:56.1596,1901-12-25 -1901-05-05 12:34:56.1597,1901-12-25 -1901-05-05 12:34:56.1598,1901-12-25 -1901-05-05 12:34:56.1599,1901-12-25 -1901-05-05 12:34:56.16,1901-12-25 -1901-05-05 12:34:56.1601,1901-12-25 -1901-05-05 12:34:56.1602,1901-12-25 -1901-05-05 12:34:56.1603,1901-12-25 -1901-05-05 12:34:56.1604,1901-12-25 -1901-05-05 12:34:56.1605,1901-12-25 -1901-05-05 12:34:56.1606,1901-12-25 -1901-05-05 12:34:56.1607,1901-12-25 -1901-05-05 12:34:56.1608,1901-12-25 -1901-05-05 12:34:56.1609,1901-12-25 -1901-05-05 12:34:56.161,1901-12-25 -1901-05-05 12:34:56.1611,1901-12-25 -1901-05-05 12:34:56.1612,1901-12-25 -1901-05-05 12:34:56.1613,1901-12-25 -1901-05-05 12:34:56.1614,1901-12-25 -1901-05-05 12:34:56.1615,1901-12-25 -1901-05-05 12:34:56.1616,1901-12-25 -1901-05-05 12:34:56.1617,1901-12-25 -1901-05-05 12:34:56.1618,1901-12-25 -1901-05-05 12:34:56.1619,1901-12-25 -1901-05-05 12:34:56.162,1901-12-25 -1901-05-05 12:34:56.1621,1901-12-25 -1901-05-05 12:34:56.1622,1901-12-25 -1901-05-05 12:34:56.1623,1901-12-25 -1901-05-05 12:34:56.1624,1901-12-25 -1901-05-05 12:34:56.1625,1901-12-25 -1901-05-05 12:34:56.1626,1901-12-25 -1901-05-05 12:34:56.1627,1901-12-25 -1901-05-05 12:34:56.1628,1901-12-25 -1901-05-05 12:34:56.1629,1901-12-25 -1901-05-05 12:34:56.163,1901-12-25 -1901-05-05 12:34:56.1631,1901-12-25 -1901-05-05 12:34:56.1632,1901-12-25 -1901-05-05 12:34:56.1633,1901-12-25 -1901-05-05 12:34:56.1634,1901-12-25 -1901-05-05 12:34:56.1635,1901-12-25 -1901-05-05 12:34:56.1636,1901-12-25 -1901-05-05 12:34:56.1637,1901-12-25 -1901-05-05 12:34:56.1638,1901-12-25 -1901-05-05 12:34:56.1639,1901-12-25 -1901-05-05 12:34:56.164,1901-12-25 -1901-05-05 12:34:56.1641,1901-12-25 -1901-05-05 12:34:56.1642,1901-12-25 -1901-05-05 12:34:56.1643,1901-12-25 -1901-05-05 12:34:56.1644,1901-12-25 -1901-05-05 12:34:56.1645,1901-12-25 -1901-05-05 12:34:56.1646,1901-12-25 -1901-05-05 12:34:56.1647,1901-12-25 -1901-05-05 12:34:56.1648,1901-12-25 -1901-05-05 12:34:56.1649,1901-12-25 -1901-05-05 12:34:56.165,1901-12-25 -1901-05-05 12:34:56.1651,1901-12-25 -1901-05-05 12:34:56.1652,1901-12-25 -1901-05-05 12:34:56.1653,1901-12-25 -1901-05-05 12:34:56.1654,1901-12-25 -1901-05-05 12:34:56.1655,1901-12-25 -1901-05-05 12:34:56.1656,1901-12-25 -1901-05-05 12:34:56.1657,1901-12-25 -1901-05-05 12:34:56.1658,1901-12-25 -1901-05-05 12:34:56.1659,1901-12-25 -1901-05-05 12:34:56.166,1901-12-25 -1901-05-05 12:34:56.1661,1901-12-25 -1901-05-05 12:34:56.1662,1901-12-25 -1901-05-05 12:34:56.1663,1901-12-25 -1901-05-05 12:34:56.1664,1901-12-25 -1901-05-05 12:34:56.1665,1901-12-25 -1901-05-05 12:34:56.1666,1901-12-25 -1901-05-05 12:34:56.1667,1901-12-25 -1901-05-05 12:34:56.1668,1901-12-25 -1901-05-05 12:34:56.1669,1901-12-25 -1901-05-05 12:34:56.167,1901-12-25 -1901-05-05 12:34:56.1671,1901-12-25 -1901-05-05 12:34:56.1672,1901-12-25 -1901-05-05 12:34:56.1673,1901-12-25 -1901-05-05 12:34:56.1674,1901-12-25 -1901-05-05 12:34:56.1675,1901-12-25 -1901-05-05 12:34:56.1676,1901-12-25 -1901-05-05 12:34:56.1677,1901-12-25 -1901-05-05 12:34:56.1678,1901-12-25 -1901-05-05 12:34:56.1679,1901-12-25 -1901-05-05 12:34:56.168,1901-12-25 -1901-05-05 12:34:56.1681,1901-12-25 -1901-05-05 12:34:56.1682,1901-12-25 -1901-05-05 12:34:56.1683,1901-12-25 -1901-05-05 12:34:56.1684,1901-12-25 -1901-05-05 12:34:56.1685,1901-12-25 -1901-05-05 12:34:56.1686,1901-12-25 -1901-05-05 12:34:56.1687,1901-12-25 -1901-05-05 12:34:56.1688,1901-12-25 -1901-05-05 12:34:56.1689,1901-12-25 -1901-05-05 12:34:56.169,1901-12-25 -1901-05-05 12:34:56.1691,1901-12-25 -1901-05-05 12:34:56.1692,1901-12-25 -1901-05-05 12:34:56.1693,1901-12-25 -1901-05-05 12:34:56.1694,1901-12-25 -1901-05-05 12:34:56.1695,1901-12-25 -1901-05-05 12:34:56.1696,1901-12-25 -1901-05-05 12:34:56.1697,1901-12-25 -1901-05-05 12:34:56.1698,1901-12-25 -1901-05-05 12:34:56.1699,1901-12-25 -1901-05-05 12:34:56.17,1901-12-25 -1901-05-05 12:34:56.1701,1901-12-25 -1901-05-05 12:34:56.1702,1901-12-25 -1901-05-05 12:34:56.1703,1901-12-25 -1901-05-05 12:34:56.1704,1901-12-25 -1901-05-05 12:34:56.1705,1901-12-25 -1901-05-05 12:34:56.1706,1901-12-25 -1901-05-05 12:34:56.1707,1901-12-25 -1901-05-05 12:34:56.1708,1901-12-25 -1901-05-05 12:34:56.1709,1901-12-25 -1901-05-05 12:34:56.171,1901-12-25 -1901-05-05 12:34:56.1711,1901-12-25 -1901-05-05 12:34:56.1712,1901-12-25 -1901-05-05 12:34:56.1713,1901-12-25 -1901-05-05 12:34:56.1714,1901-12-25 -1901-05-05 12:34:56.1715,1901-12-25 -1901-05-05 12:34:56.1716,1901-12-25 -1901-05-05 12:34:56.1717,1901-12-25 -1901-05-05 12:34:56.1718,1901-12-25 -1901-05-05 12:34:56.1719,1901-12-25 -1901-05-05 12:34:56.172,1901-12-25 -1901-05-05 12:34:56.1721,1901-12-25 -1901-05-05 12:34:56.1722,1901-12-25 -1901-05-05 12:34:56.1723,1901-12-25 -1901-05-05 12:34:56.1724,1901-12-25 -1901-05-05 12:34:56.1725,1901-12-25 -1901-05-05 12:34:56.1726,1901-12-25 -1901-05-05 12:34:56.1727,1901-12-25 -1901-05-05 12:34:56.1728,1901-12-25 -1901-05-05 12:34:56.1729,1901-12-25 -1901-05-05 12:34:56.173,1901-12-25 -1901-05-05 12:34:56.1731,1901-12-25 -1901-05-05 12:34:56.1732,1901-12-25 -1901-05-05 12:34:56.1733,1901-12-25 -1901-05-05 12:34:56.1734,1901-12-25 -1901-05-05 12:34:56.1735,1901-12-25 -1901-05-05 12:34:56.1736,1901-12-25 -1901-05-05 12:34:56.1737,1901-12-25 -1901-05-05 12:34:56.1738,1901-12-25 -1901-05-05 12:34:56.1739,1901-12-25 -1901-05-05 12:34:56.174,1901-12-25 -1901-05-05 12:34:56.1741,1901-12-25 -1901-05-05 12:34:56.1742,1901-12-25 -1901-05-05 12:34:56.1743,1901-12-25 -1901-05-05 12:34:56.1744,1901-12-25 -1901-05-05 12:34:56.1745,1901-12-25 -1901-05-05 12:34:56.1746,1901-12-25 -1901-05-05 12:34:56.1747,1901-12-25 -1901-05-05 12:34:56.1748,1901-12-25 -1901-05-05 12:34:56.1749,1901-12-25 -1901-05-05 12:34:56.175,1901-12-25 -1901-05-05 12:34:56.1751,1901-12-25 -1901-05-05 12:34:56.1752,1901-12-25 -1901-05-05 12:34:56.1753,1901-12-25 -1901-05-05 12:34:56.1754,1901-12-25 -1901-05-05 12:34:56.1755,1901-12-25 -1901-05-05 12:34:56.1756,1901-12-25 -1901-05-05 12:34:56.1757,1901-12-25 -1901-05-05 12:34:56.1758,1901-12-25 -1901-05-05 12:34:56.1759,1901-12-25 -1901-05-05 12:34:56.176,1901-12-25 -1901-05-05 12:34:56.1761,1901-12-25 -1901-05-05 12:34:56.1762,1901-12-25 -1901-05-05 12:34:56.1763,1901-12-25 -1901-05-05 12:34:56.1764,1901-12-25 -1901-05-05 12:34:56.1765,1901-12-25 -1901-05-05 12:34:56.1766,1901-12-25 -1901-05-05 12:34:56.1767,1901-12-25 -1901-05-05 12:34:56.1768,1901-12-25 -1901-05-05 12:34:56.1769,1901-12-25 -1901-05-05 12:34:56.177,1901-12-25 -1901-05-05 12:34:56.1771,1901-12-25 -1901-05-05 12:34:56.1772,1901-12-25 -1901-05-05 12:34:56.1773,1901-12-25 -1901-05-05 12:34:56.1774,1901-12-25 -1901-05-05 12:34:56.1775,1901-12-25 -1901-05-05 12:34:56.1776,1901-12-25 -1901-05-05 12:34:56.1777,1901-12-25 -1901-05-05 12:34:56.1778,1901-12-25 -1901-05-05 12:34:56.1779,1901-12-25 -1901-05-05 12:34:56.178,1901-12-25 -1901-05-05 12:34:56.1781,1901-12-25 -1901-05-05 12:34:56.1782,1901-12-25 -1901-05-05 12:34:56.1783,1901-12-25 -1901-05-05 12:34:56.1784,1901-12-25 -1901-05-05 12:34:56.1785,1901-12-25 -1901-05-05 12:34:56.1786,1901-12-25 -1901-05-05 12:34:56.1787,1901-12-25 -1901-05-05 12:34:56.1788,1901-12-25 -1901-05-05 12:34:56.1789,1901-12-25 -1901-05-05 12:34:56.179,1901-12-25 -1901-05-05 12:34:56.1791,1901-12-25 -1901-05-05 12:34:56.1792,1901-12-25 -1901-05-05 12:34:56.1793,1901-12-25 -1901-05-05 12:34:56.1794,1901-12-25 -1901-05-05 12:34:56.1795,1901-12-25 -1901-05-05 12:34:56.1796,1901-12-25 -1901-05-05 12:34:56.1797,1901-12-25 -1901-05-05 12:34:56.1798,1901-12-25 -1901-05-05 12:34:56.1799,1901-12-25 -1901-05-05 12:34:56.18,1901-12-25 -1901-05-05 12:34:56.1801,1901-12-25 -1901-05-05 12:34:56.1802,1901-12-25 -1901-05-05 12:34:56.1803,1901-12-25 -1901-05-05 12:34:56.1804,1901-12-25 -1901-05-05 12:34:56.1805,1901-12-25 -1901-05-05 12:34:56.1806,1901-12-25 -1901-05-05 12:34:56.1807,1901-12-25 -1901-05-05 12:34:56.1808,1901-12-25 -1901-05-05 12:34:56.1809,1901-12-25 -1901-05-05 12:34:56.181,1901-12-25 -1901-05-05 12:34:56.1811,1901-12-25 -1901-05-05 12:34:56.1812,1901-12-25 -1901-05-05 12:34:56.1813,1901-12-25 -1901-05-05 12:34:56.1814,1901-12-25 -1901-05-05 12:34:56.1815,1901-12-25 -1901-05-05 12:34:56.1816,1901-12-25 -1901-05-05 12:34:56.1817,1901-12-25 -1901-05-05 12:34:56.1818,1901-12-25 -1901-05-05 12:34:56.1819,1901-12-25 -1901-05-05 12:34:56.182,1901-12-25 -1901-05-05 12:34:56.1821,1901-12-25 -1901-05-05 12:34:56.1822,1901-12-25 -1901-05-05 12:34:56.1823,1901-12-25 -1901-05-05 12:34:56.1824,1901-12-25 -1901-05-05 12:34:56.1825,1901-12-25 -1901-05-05 12:34:56.1826,1901-12-25 -1901-05-05 12:34:56.1827,1901-12-25 -1901-05-05 12:34:56.1828,1901-12-25 -1901-05-05 12:34:56.1829,1901-12-25 -1901-05-05 12:34:56.183,1901-12-25 -1901-05-05 12:34:56.1831,1901-12-25 -1901-05-05 12:34:56.1832,1901-12-25 -1901-05-05 12:34:56.1833,1901-12-25 -1901-05-05 12:34:56.1834,1901-12-25 -1901-05-05 12:34:56.1835,1901-12-25 -1901-05-05 12:34:56.1836,1901-12-25 -1901-05-05 12:34:56.1837,1901-12-25 -1901-05-05 12:34:56.1838,1901-12-25 -1901-05-05 12:34:56.1839,1901-12-25 -1901-05-05 12:34:56.184,1901-12-25 -1901-05-05 12:34:56.1841,1901-12-25 -1901-05-05 12:34:56.1842,1901-12-25 -1901-05-05 12:34:56.1843,1901-12-25 -1901-05-05 12:34:56.1844,1901-12-25 -1901-05-05 12:34:56.1845,1901-12-25 -1901-05-05 12:34:56.1846,1901-12-25 -1901-05-05 12:34:56.1847,1901-12-25 -1901-05-05 12:34:56.1848,1901-12-25 -1901-05-05 12:34:56.1849,1901-12-25 -1901-05-05 12:34:56.185,1901-12-25 -1901-05-05 12:34:56.1851,1901-12-25 -1901-05-05 12:34:56.1852,1901-12-25 -1901-05-05 12:34:56.1853,1901-12-25 -1901-05-05 12:34:56.1854,1901-12-25 -1901-05-05 12:34:56.1855,1901-12-25 -1901-05-05 12:34:56.1856,1901-12-25 -1901-05-05 12:34:56.1857,1901-12-25 -1901-05-05 12:34:56.1858,1901-12-25 -1901-05-05 12:34:56.1859,1901-12-25 -1901-05-05 12:34:56.186,1901-12-25 -1901-05-05 12:34:56.1861,1901-12-25 -1901-05-05 12:34:56.1862,1901-12-25 -1901-05-05 12:34:56.1863,1901-12-25 -1901-05-05 12:34:56.1864,1901-12-25 -1901-05-05 12:34:56.1865,1901-12-25 -1901-05-05 12:34:56.1866,1901-12-25 -1901-05-05 12:34:56.1867,1901-12-25 -1901-05-05 12:34:56.1868,1901-12-25 -1901-05-05 12:34:56.1869,1901-12-25 -1901-05-05 12:34:56.187,1901-12-25 -1901-05-05 12:34:56.1871,1901-12-25 -1901-05-05 12:34:56.1872,1901-12-25 -1901-05-05 12:34:56.1873,1901-12-25 -1901-05-05 12:34:56.1874,1901-12-25 -1901-05-05 12:34:56.1875,1901-12-25 -1901-05-05 12:34:56.1876,1901-12-25 -1901-05-05 12:34:56.1877,1901-12-25 -1901-05-05 12:34:56.1878,1901-12-25 -1901-05-05 12:34:56.1879,1901-12-25 -1901-05-05 12:34:56.188,1901-12-25 -1901-05-05 12:34:56.1881,1901-12-25 -1901-05-05 12:34:56.1882,1901-12-25 -1901-05-05 12:34:56.1883,1901-12-25 -1901-05-05 12:34:56.1884,1901-12-25 -1901-05-05 12:34:56.1885,1901-12-25 -1901-05-05 12:34:56.1886,1901-12-25 -1901-05-05 12:34:56.1887,1901-12-25 -1901-05-05 12:34:56.1888,1901-12-25 -1901-05-05 12:34:56.1889,1901-12-25 -1901-05-05 12:34:56.189,1901-12-25 -1901-05-05 12:34:56.1891,1901-12-25 -1901-05-05 12:34:56.1892,1901-12-25 -1901-05-05 12:34:56.1893,1901-12-25 -1901-05-05 12:34:56.1894,1901-12-25 -1901-05-05 12:34:56.1895,1901-12-25 -1901-05-05 12:34:56.1896,1901-12-25 -1901-05-05 12:34:56.1897,1901-12-25 -1901-05-05 12:34:56.1898,1901-12-25 -1901-05-05 12:34:56.1899,1901-12-25 -1901-05-05 12:34:56.19,1901-12-25 -1901-05-05 12:34:56.1901,1901-12-25 -1901-05-05 12:34:56.1902,1901-12-25 -1901-05-05 12:34:56.1903,1901-12-25 -1901-05-05 12:34:56.1904,1901-12-25 -1901-05-05 12:34:56.1905,1901-12-25 -1901-05-05 12:34:56.1906,1901-12-25 -1901-05-05 12:34:56.1907,1901-12-25 -1901-05-05 12:34:56.1908,1901-12-25 -1901-05-05 12:34:56.1909,1901-12-25 -1901-05-05 12:34:56.191,1901-12-25 -1901-05-05 12:34:56.1911,1901-12-25 -1901-05-05 12:34:56.1912,1901-12-25 -1901-05-05 12:34:56.1913,1901-12-25 -1901-05-05 12:34:56.1914,1901-12-25 -1901-05-05 12:34:56.1915,1901-12-25 -1901-05-05 12:34:56.1916,1901-12-25 -1901-05-05 12:34:56.1917,1901-12-25 -1901-05-05 12:34:56.1918,1901-12-25 -1901-05-05 12:34:56.1919,1901-12-25 -1901-05-05 12:34:56.192,1901-12-25 -1901-05-05 12:34:56.1921,1901-12-25 -1901-05-05 12:34:56.1922,1901-12-25 -1901-05-05 12:34:56.1923,1901-12-25 -1901-05-05 12:34:56.1924,1901-12-25 -1901-05-05 12:34:56.1925,1901-12-25 -1901-05-05 12:34:56.1926,1901-12-25 -1901-05-05 12:34:56.1927,1901-12-25 -1901-05-05 12:34:56.1928,1901-12-25 -1901-05-05 12:34:56.1929,1901-12-25 -1901-05-05 12:34:56.193,1901-12-25 -1901-05-05 12:34:56.1931,1901-12-25 -1901-05-05 12:34:56.1932,1901-12-25 -1901-05-05 12:34:56.1933,1901-12-25 -1901-05-05 12:34:56.1934,1901-12-25 -1901-05-05 12:34:56.1935,1901-12-25 -1901-05-05 12:34:56.1936,1901-12-25 -1901-05-05 12:34:56.1937,1901-12-25 -1901-05-05 12:34:56.1938,1901-12-25 -1901-05-05 12:34:56.1939,1901-12-25 -1901-05-05 12:34:56.194,1901-12-25 -1901-05-05 12:34:56.1941,1901-12-25 -1901-05-05 12:34:56.1942,1901-12-25 -1901-05-05 12:34:56.1943,1901-12-25 -1901-05-05 12:34:56.1944,1901-12-25 -1901-05-05 12:34:56.1945,1901-12-25 -1901-05-05 12:34:56.1946,1901-12-25 -1901-05-05 12:34:56.1947,1901-12-25 -1901-05-05 12:34:56.1948,1901-12-25 -1901-05-05 12:34:56.1949,1901-12-25 -1901-05-05 12:34:56.195,1901-12-25 -1901-05-05 12:34:56.1951,1901-12-25 -1901-05-05 12:34:56.1952,1901-12-25 -1901-05-05 12:34:56.1953,1901-12-25 -1901-05-05 12:34:56.1954,1901-12-25 -1901-05-05 12:34:56.1955,1901-12-25 -1901-05-05 12:34:56.1956,1901-12-25 -1901-05-05 12:34:56.1957,1901-12-25 -1901-05-05 12:34:56.1958,1901-12-25 -1901-05-05 12:34:56.1959,1901-12-25 -1901-05-05 12:34:56.196,1901-12-25 -1901-05-05 12:34:56.1961,1901-12-25 -1901-05-05 12:34:56.1962,1901-12-25 -1901-05-05 12:34:56.1963,1901-12-25 -1901-05-05 12:34:56.1964,1901-12-25 -1901-05-05 12:34:56.1965,1901-12-25 -1901-05-05 12:34:56.1966,1901-12-25 -1901-05-05 12:34:56.1967,1901-12-25 -1901-05-05 12:34:56.1968,1901-12-25 -1901-05-05 12:34:56.1969,1901-12-25 -1901-05-05 12:34:56.197,1901-12-25 -1901-05-05 12:34:56.1971,1901-12-25 -1901-05-05 12:34:56.1972,1901-12-25 -1901-05-05 12:34:56.1973,1901-12-25 -1901-05-05 12:34:56.1974,1901-12-25 -1901-05-05 12:34:56.1975,1901-12-25 -1901-05-05 12:34:56.1976,1901-12-25 -1901-05-05 12:34:56.1977,1901-12-25 -1901-05-05 12:34:56.1978,1901-12-25 -1901-05-05 12:34:56.1979,1901-12-25 -1901-05-05 12:34:56.198,1901-12-25 -1901-05-05 12:34:56.1981,1901-12-25 -1901-05-05 12:34:56.1982,1901-12-25 -1901-05-05 12:34:56.1983,1901-12-25 -1901-05-05 12:34:56.1984,1901-12-25 -1901-05-05 12:34:56.1985,1901-12-25 -1901-05-05 12:34:56.1986,1901-12-25 -1901-05-05 12:34:56.1987,1901-12-25 -1901-05-05 12:34:56.1988,1901-12-25 -1901-05-05 12:34:56.1989,1901-12-25 -1901-05-05 12:34:56.199,1901-12-25 -1901-05-05 12:34:56.1991,1901-12-25 -1901-05-05 12:34:56.1992,1901-12-25 -1901-05-05 12:34:56.1993,1901-12-25 -1901-05-05 12:34:56.1994,1901-12-25 -1901-05-05 12:34:56.1995,1901-12-25 -1901-05-05 12:34:56.1996,1901-12-25 -1901-05-05 12:34:56.1997,1901-12-25 -1901-05-05 12:34:56.1998,1901-12-25 -1901-05-05 12:34:56.1999,1901-12-25 -1902-05-05 12:34:56.1,1902-12-25 -1902-05-05 12:34:56.1001,1902-12-25 -1902-05-05 12:34:56.1002,1902-12-25 -1902-05-05 12:34:56.1003,1902-12-25 -1902-05-05 12:34:56.1004,1902-12-25 -1902-05-05 12:34:56.1005,1902-12-25 -1902-05-05 12:34:56.1006,1902-12-25 -1902-05-05 12:34:56.1007,1902-12-25 -1902-05-05 12:34:56.1008,1902-12-25 -1902-05-05 12:34:56.1009,1902-12-25 -1902-05-05 12:34:56.101,1902-12-25 -1902-05-05 12:34:56.1011,1902-12-25 -1902-05-05 12:34:56.1012,1902-12-25 -1902-05-05 12:34:56.1013,1902-12-25 -1902-05-05 12:34:56.1014,1902-12-25 -1902-05-05 12:34:56.1015,1902-12-25 -1902-05-05 12:34:56.1016,1902-12-25 -1902-05-05 12:34:56.1017,1902-12-25 -1902-05-05 12:34:56.1018,1902-12-25 -1902-05-05 12:34:56.1019,1902-12-25 -1902-05-05 12:34:56.102,1902-12-25 -1902-05-05 12:34:56.1021,1902-12-25 -1902-05-05 12:34:56.1022,1902-12-25 -1902-05-05 12:34:56.1023,1902-12-25 -1902-05-05 12:34:56.1024,1902-12-25 -1902-05-05 12:34:56.1025,1902-12-25 -1902-05-05 12:34:56.1026,1902-12-25 -1902-05-05 12:34:56.1027,1902-12-25 -1902-05-05 12:34:56.1028,1902-12-25 -1902-05-05 12:34:56.1029,1902-12-25 -1902-05-05 12:34:56.103,1902-12-25 -1902-05-05 12:34:56.1031,1902-12-25 -1902-05-05 12:34:56.1032,1902-12-25 -1902-05-05 12:34:56.1033,1902-12-25 -1902-05-05 12:34:56.1034,1902-12-25 -1902-05-05 12:34:56.1035,1902-12-25 -1902-05-05 12:34:56.1036,1902-12-25 -1902-05-05 12:34:56.1037,1902-12-25 -1902-05-05 12:34:56.1038,1902-12-25 -1902-05-05 12:34:56.1039,1902-12-25 -1902-05-05 12:34:56.104,1902-12-25 -1902-05-05 12:34:56.1041,1902-12-25 -1902-05-05 12:34:56.1042,1902-12-25 -1902-05-05 12:34:56.1043,1902-12-25 -1902-05-05 12:34:56.1044,1902-12-25 -1902-05-05 12:34:56.1045,1902-12-25 -1902-05-05 12:34:56.1046,1902-12-25 -1902-05-05 12:34:56.1047,1902-12-25 -1902-05-05 12:34:56.1048,1902-12-25 -1902-05-05 12:34:56.1049,1902-12-25 -1902-05-05 12:34:56.105,1902-12-25 -1902-05-05 12:34:56.1051,1902-12-25 -1902-05-05 12:34:56.1052,1902-12-25 -1902-05-05 12:34:56.1053,1902-12-25 -1902-05-05 12:34:56.1054,1902-12-25 -1902-05-05 12:34:56.1055,1902-12-25 -1902-05-05 12:34:56.1056,1902-12-25 -1902-05-05 12:34:56.1057,1902-12-25 -1902-05-05 12:34:56.1058,1902-12-25 -1902-05-05 12:34:56.1059,1902-12-25 -1902-05-05 12:34:56.106,1902-12-25 -1902-05-05 12:34:56.1061,1902-12-25 -1902-05-05 12:34:56.1062,1902-12-25 -1902-05-05 12:34:56.1063,1902-12-25 -1902-05-05 12:34:56.1064,1902-12-25 -1902-05-05 12:34:56.1065,1902-12-25 -1902-05-05 12:34:56.1066,1902-12-25 -1902-05-05 12:34:56.1067,1902-12-25 -1902-05-05 12:34:56.1068,1902-12-25 -1902-05-05 12:34:56.1069,1902-12-25 -1902-05-05 12:34:56.107,1902-12-25 -1902-05-05 12:34:56.1071,1902-12-25 -1902-05-05 12:34:56.1072,1902-12-25 -1902-05-05 12:34:56.1073,1902-12-25 -1902-05-05 12:34:56.1074,1902-12-25 -1902-05-05 12:34:56.1075,1902-12-25 -1902-05-05 12:34:56.1076,1902-12-25 -1902-05-05 12:34:56.1077,1902-12-25 -1902-05-05 12:34:56.1078,1902-12-25 -1902-05-05 12:34:56.1079,1902-12-25 -1902-05-05 12:34:56.108,1902-12-25 -1902-05-05 12:34:56.1081,1902-12-25 -1902-05-05 12:34:56.1082,1902-12-25 -1902-05-05 12:34:56.1083,1902-12-25 -1902-05-05 12:34:56.1084,1902-12-25 -1902-05-05 12:34:56.1085,1902-12-25 -1902-05-05 12:34:56.1086,1902-12-25 -1902-05-05 12:34:56.1087,1902-12-25 -1902-05-05 12:34:56.1088,1902-12-25 -1902-05-05 12:34:56.1089,1902-12-25 -1902-05-05 12:34:56.109,1902-12-25 -1902-05-05 12:34:56.1091,1902-12-25 -1902-05-05 12:34:56.1092,1902-12-25 -1902-05-05 12:34:56.1093,1902-12-25 -1902-05-05 12:34:56.1094,1902-12-25 -1902-05-05 12:34:56.1095,1902-12-25 -1902-05-05 12:34:56.1096,1902-12-25 -1902-05-05 12:34:56.1097,1902-12-25 -1902-05-05 12:34:56.1098,1902-12-25 -1902-05-05 12:34:56.1099,1902-12-25 -1902-05-05 12:34:56.11,1902-12-25 -1902-05-05 12:34:56.1101,1902-12-25 -1902-05-05 12:34:56.1102,1902-12-25 -1902-05-05 12:34:56.1103,1902-12-25 -1902-05-05 12:34:56.1104,1902-12-25 -1902-05-05 12:34:56.1105,1902-12-25 -1902-05-05 12:34:56.1106,1902-12-25 -1902-05-05 12:34:56.1107,1902-12-25 -1902-05-05 12:34:56.1108,1902-12-25 -1902-05-05 12:34:56.1109,1902-12-25 -1902-05-05 12:34:56.111,1902-12-25 -1902-05-05 12:34:56.1111,1902-12-25 -1902-05-05 12:34:56.1112,1902-12-25 -1902-05-05 12:34:56.1113,1902-12-25 -1902-05-05 12:34:56.1114,1902-12-25 -1902-05-05 12:34:56.1115,1902-12-25 -1902-05-05 12:34:56.1116,1902-12-25 -1902-05-05 12:34:56.1117,1902-12-25 -1902-05-05 12:34:56.1118,1902-12-25 -1902-05-05 12:34:56.1119,1902-12-25 -1902-05-05 12:34:56.112,1902-12-25 -1902-05-05 12:34:56.1121,1902-12-25 -1902-05-05 12:34:56.1122,1902-12-25 -1902-05-05 12:34:56.1123,1902-12-25 -1902-05-05 12:34:56.1124,1902-12-25 -1902-05-05 12:34:56.1125,1902-12-25 -1902-05-05 12:34:56.1126,1902-12-25 -1902-05-05 12:34:56.1127,1902-12-25 -1902-05-05 12:34:56.1128,1902-12-25 -1902-05-05 12:34:56.1129,1902-12-25 -1902-05-05 12:34:56.113,1902-12-25 -1902-05-05 12:34:56.1131,1902-12-25 -1902-05-05 12:34:56.1132,1902-12-25 -1902-05-05 12:34:56.1133,1902-12-25 -1902-05-05 12:34:56.1134,1902-12-25 -1902-05-05 12:34:56.1135,1902-12-25 -1902-05-05 12:34:56.1136,1902-12-25 -1902-05-05 12:34:56.1137,1902-12-25 -1902-05-05 12:34:56.1138,1902-12-25 -1902-05-05 12:34:56.1139,1902-12-25 -1902-05-05 12:34:56.114,1902-12-25 -1902-05-05 12:34:56.1141,1902-12-25 -1902-05-05 12:34:56.1142,1902-12-25 -1902-05-05 12:34:56.1143,1902-12-25 -1902-05-05 12:34:56.1144,1902-12-25 -1902-05-05 12:34:56.1145,1902-12-25 -1902-05-05 12:34:56.1146,1902-12-25 -1902-05-05 12:34:56.1147,1902-12-25 -1902-05-05 12:34:56.1148,1902-12-25 -1902-05-05 12:34:56.1149,1902-12-25 -1902-05-05 12:34:56.115,1902-12-25 -1902-05-05 12:34:56.1151,1902-12-25 -1902-05-05 12:34:56.1152,1902-12-25 -1902-05-05 12:34:56.1153,1902-12-25 -1902-05-05 12:34:56.1154,1902-12-25 -1902-05-05 12:34:56.1155,1902-12-25 -1902-05-05 12:34:56.1156,1902-12-25 -1902-05-05 12:34:56.1157,1902-12-25 -1902-05-05 12:34:56.1158,1902-12-25 -1902-05-05 12:34:56.1159,1902-12-25 -1902-05-05 12:34:56.116,1902-12-25 -1902-05-05 12:34:56.1161,1902-12-25 -1902-05-05 12:34:56.1162,1902-12-25 -1902-05-05 12:34:56.1163,1902-12-25 -1902-05-05 12:34:56.1164,1902-12-25 -1902-05-05 12:34:56.1165,1902-12-25 -1902-05-05 12:34:56.1166,1902-12-25 -1902-05-05 12:34:56.1167,1902-12-25 -1902-05-05 12:34:56.1168,1902-12-25 -1902-05-05 12:34:56.1169,1902-12-25 -1902-05-05 12:34:56.117,1902-12-25 -1902-05-05 12:34:56.1171,1902-12-25 -1902-05-05 12:34:56.1172,1902-12-25 -1902-05-05 12:34:56.1173,1902-12-25 -1902-05-05 12:34:56.1174,1902-12-25 -1902-05-05 12:34:56.1175,1902-12-25 -1902-05-05 12:34:56.1176,1902-12-25 -1902-05-05 12:34:56.1177,1902-12-25 -1902-05-05 12:34:56.1178,1902-12-25 -1902-05-05 12:34:56.1179,1902-12-25 -1902-05-05 12:34:56.118,1902-12-25 -1902-05-05 12:34:56.1181,1902-12-25 -1902-05-05 12:34:56.1182,1902-12-25 -1902-05-05 12:34:56.1183,1902-12-25 -1902-05-05 12:34:56.1184,1902-12-25 -1902-05-05 12:34:56.1185,1902-12-25 -1902-05-05 12:34:56.1186,1902-12-25 -1902-05-05 12:34:56.1187,1902-12-25 -1902-05-05 12:34:56.1188,1902-12-25 -1902-05-05 12:34:56.1189,1902-12-25 -1902-05-05 12:34:56.119,1902-12-25 -1902-05-05 12:34:56.1191,1902-12-25 -1902-05-05 12:34:56.1192,1902-12-25 -1902-05-05 12:34:56.1193,1902-12-25 -1902-05-05 12:34:56.1194,1902-12-25 -1902-05-05 12:34:56.1195,1902-12-25 -1902-05-05 12:34:56.1196,1902-12-25 -1902-05-05 12:34:56.1197,1902-12-25 -1902-05-05 12:34:56.1198,1902-12-25 -1902-05-05 12:34:56.1199,1902-12-25 -1902-05-05 12:34:56.12,1902-12-25 -1902-05-05 12:34:56.1201,1902-12-25 -1902-05-05 12:34:56.1202,1902-12-25 -1902-05-05 12:34:56.1203,1902-12-25 -1902-05-05 12:34:56.1204,1902-12-25 -1902-05-05 12:34:56.1205,1902-12-25 -1902-05-05 12:34:56.1206,1902-12-25 -1902-05-05 12:34:56.1207,1902-12-25 -1902-05-05 12:34:56.1208,1902-12-25 -1902-05-05 12:34:56.1209,1902-12-25 -1902-05-05 12:34:56.121,1902-12-25 -1902-05-05 12:34:56.1211,1902-12-25 -1902-05-05 12:34:56.1212,1902-12-25 -1902-05-05 12:34:56.1213,1902-12-25 -1902-05-05 12:34:56.1214,1902-12-25 -1902-05-05 12:34:56.1215,1902-12-25 -1902-05-05 12:34:56.1216,1902-12-25 -1902-05-05 12:34:56.1217,1902-12-25 -1902-05-05 12:34:56.1218,1902-12-25 -1902-05-05 12:34:56.1219,1902-12-25 -1902-05-05 12:34:56.122,1902-12-25 -1902-05-05 12:34:56.1221,1902-12-25 -1902-05-05 12:34:56.1222,1902-12-25 -1902-05-05 12:34:56.1223,1902-12-25 -1902-05-05 12:34:56.1224,1902-12-25 -1902-05-05 12:34:56.1225,1902-12-25 -1902-05-05 12:34:56.1226,1902-12-25 -1902-05-05 12:34:56.1227,1902-12-25 -1902-05-05 12:34:56.1228,1902-12-25 -1902-05-05 12:34:56.1229,1902-12-25 -1902-05-05 12:34:56.123,1902-12-25 -1902-05-05 12:34:56.1231,1902-12-25 -1902-05-05 12:34:56.1232,1902-12-25 -1902-05-05 12:34:56.1233,1902-12-25 -1902-05-05 12:34:56.1234,1902-12-25 -1902-05-05 12:34:56.1235,1902-12-25 -1902-05-05 12:34:56.1236,1902-12-25 -1902-05-05 12:34:56.1237,1902-12-25 -1902-05-05 12:34:56.1238,1902-12-25 -1902-05-05 12:34:56.1239,1902-12-25 -1902-05-05 12:34:56.124,1902-12-25 -1902-05-05 12:34:56.1241,1902-12-25 -1902-05-05 12:34:56.1242,1902-12-25 -1902-05-05 12:34:56.1243,1902-12-25 -1902-05-05 12:34:56.1244,1902-12-25 -1902-05-05 12:34:56.1245,1902-12-25 -1902-05-05 12:34:56.1246,1902-12-25 -1902-05-05 12:34:56.1247,1902-12-25 -1902-05-05 12:34:56.1248,1902-12-25 -1902-05-05 12:34:56.1249,1902-12-25 -1902-05-05 12:34:56.125,1902-12-25 -1902-05-05 12:34:56.1251,1902-12-25 -1902-05-05 12:34:56.1252,1902-12-25 -1902-05-05 12:34:56.1253,1902-12-25 -1902-05-05 12:34:56.1254,1902-12-25 -1902-05-05 12:34:56.1255,1902-12-25 -1902-05-05 12:34:56.1256,1902-12-25 -1902-05-05 12:34:56.1257,1902-12-25 -1902-05-05 12:34:56.1258,1902-12-25 -1902-05-05 12:34:56.1259,1902-12-25 -1902-05-05 12:34:56.126,1902-12-25 -1902-05-05 12:34:56.1261,1902-12-25 -1902-05-05 12:34:56.1262,1902-12-25 -1902-05-05 12:34:56.1263,1902-12-25 -1902-05-05 12:34:56.1264,1902-12-25 -1902-05-05 12:34:56.1265,1902-12-25 -1902-05-05 12:34:56.1266,1902-12-25 -1902-05-05 12:34:56.1267,1902-12-25 -1902-05-05 12:34:56.1268,1902-12-25 -1902-05-05 12:34:56.1269,1902-12-25 -1902-05-05 12:34:56.127,1902-12-25 -1902-05-05 12:34:56.1271,1902-12-25 -1902-05-05 12:34:56.1272,1902-12-25 -1902-05-05 12:34:56.1273,1902-12-25 -1902-05-05 12:34:56.1274,1902-12-25 -1902-05-05 12:34:56.1275,1902-12-25 -1902-05-05 12:34:56.1276,1902-12-25 -1902-05-05 12:34:56.1277,1902-12-25 -1902-05-05 12:34:56.1278,1902-12-25 -1902-05-05 12:34:56.1279,1902-12-25 -1902-05-05 12:34:56.128,1902-12-25 -1902-05-05 12:34:56.1281,1902-12-25 -1902-05-05 12:34:56.1282,1902-12-25 -1902-05-05 12:34:56.1283,1902-12-25 -1902-05-05 12:34:56.1284,1902-12-25 -1902-05-05 12:34:56.1285,1902-12-25 -1902-05-05 12:34:56.1286,1902-12-25 -1902-05-05 12:34:56.1287,1902-12-25 -1902-05-05 12:34:56.1288,1902-12-25 -1902-05-05 12:34:56.1289,1902-12-25 -1902-05-05 12:34:56.129,1902-12-25 -1902-05-05 12:34:56.1291,1902-12-25 -1902-05-05 12:34:56.1292,1902-12-25 -1902-05-05 12:34:56.1293,1902-12-25 -1902-05-05 12:34:56.1294,1902-12-25 -1902-05-05 12:34:56.1295,1902-12-25 -1902-05-05 12:34:56.1296,1902-12-25 -1902-05-05 12:34:56.1297,1902-12-25 -1902-05-05 12:34:56.1298,1902-12-25 -1902-05-05 12:34:56.1299,1902-12-25 -1902-05-05 12:34:56.13,1902-12-25 -1902-05-05 12:34:56.1301,1902-12-25 -1902-05-05 12:34:56.1302,1902-12-25 -1902-05-05 12:34:56.1303,1902-12-25 -1902-05-05 12:34:56.1304,1902-12-25 -1902-05-05 12:34:56.1305,1902-12-25 -1902-05-05 12:34:56.1306,1902-12-25 -1902-05-05 12:34:56.1307,1902-12-25 -1902-05-05 12:34:56.1308,1902-12-25 -1902-05-05 12:34:56.1309,1902-12-25 -1902-05-05 12:34:56.131,1902-12-25 -1902-05-05 12:34:56.1311,1902-12-25 -1902-05-05 12:34:56.1312,1902-12-25 -1902-05-05 12:34:56.1313,1902-12-25 -1902-05-05 12:34:56.1314,1902-12-25 -1902-05-05 12:34:56.1315,1902-12-25 -1902-05-05 12:34:56.1316,1902-12-25 -1902-05-05 12:34:56.1317,1902-12-25 -1902-05-05 12:34:56.1318,1902-12-25 -1902-05-05 12:34:56.1319,1902-12-25 -1902-05-05 12:34:56.132,1902-12-25 -1902-05-05 12:34:56.1321,1902-12-25 -1902-05-05 12:34:56.1322,1902-12-25 -1902-05-05 12:34:56.1323,1902-12-25 -1902-05-05 12:34:56.1324,1902-12-25 -1902-05-05 12:34:56.1325,1902-12-25 -1902-05-05 12:34:56.1326,1902-12-25 -1902-05-05 12:34:56.1327,1902-12-25 -1902-05-05 12:34:56.1328,1902-12-25 -1902-05-05 12:34:56.1329,1902-12-25 -1902-05-05 12:34:56.133,1902-12-25 -1902-05-05 12:34:56.1331,1902-12-25 -1902-05-05 12:34:56.1332,1902-12-25 -1902-05-05 12:34:56.1333,1902-12-25 -1902-05-05 12:34:56.1334,1902-12-25 -1902-05-05 12:34:56.1335,1902-12-25 -1902-05-05 12:34:56.1336,1902-12-25 -1902-05-05 12:34:56.1337,1902-12-25 -1902-05-05 12:34:56.1338,1902-12-25 -1902-05-05 12:34:56.1339,1902-12-25 -1902-05-05 12:34:56.134,1902-12-25 -1902-05-05 12:34:56.1341,1902-12-25 -1902-05-05 12:34:56.1342,1902-12-25 -1902-05-05 12:34:56.1343,1902-12-25 -1902-05-05 12:34:56.1344,1902-12-25 -1902-05-05 12:34:56.1345,1902-12-25 -1902-05-05 12:34:56.1346,1902-12-25 -1902-05-05 12:34:56.1347,1902-12-25 -1902-05-05 12:34:56.1348,1902-12-25 -1902-05-05 12:34:56.1349,1902-12-25 -1902-05-05 12:34:56.135,1902-12-25 -1902-05-05 12:34:56.1351,1902-12-25 -1902-05-05 12:34:56.1352,1902-12-25 -1902-05-05 12:34:56.1353,1902-12-25 -1902-05-05 12:34:56.1354,1902-12-25 -1902-05-05 12:34:56.1355,1902-12-25 -1902-05-05 12:34:56.1356,1902-12-25 -1902-05-05 12:34:56.1357,1902-12-25 -1902-05-05 12:34:56.1358,1902-12-25 -1902-05-05 12:34:56.1359,1902-12-25 -1902-05-05 12:34:56.136,1902-12-25 -1902-05-05 12:34:56.1361,1902-12-25 -1902-05-05 12:34:56.1362,1902-12-25 -1902-05-05 12:34:56.1363,1902-12-25 -1902-05-05 12:34:56.1364,1902-12-25 -1902-05-05 12:34:56.1365,1902-12-25 -1902-05-05 12:34:56.1366,1902-12-25 -1902-05-05 12:34:56.1367,1902-12-25 -1902-05-05 12:34:56.1368,1902-12-25 -1902-05-05 12:34:56.1369,1902-12-25 -1902-05-05 12:34:56.137,1902-12-25 -1902-05-05 12:34:56.1371,1902-12-25 -1902-05-05 12:34:56.1372,1902-12-25 -1902-05-05 12:34:56.1373,1902-12-25 -1902-05-05 12:34:56.1374,1902-12-25 -1902-05-05 12:34:56.1375,1902-12-25 -1902-05-05 12:34:56.1376,1902-12-25 -1902-05-05 12:34:56.1377,1902-12-25 -1902-05-05 12:34:56.1378,1902-12-25 -1902-05-05 12:34:56.1379,1902-12-25 -1902-05-05 12:34:56.138,1902-12-25 -1902-05-05 12:34:56.1381,1902-12-25 -1902-05-05 12:34:56.1382,1902-12-25 -1902-05-05 12:34:56.1383,1902-12-25 -1902-05-05 12:34:56.1384,1902-12-25 -1902-05-05 12:34:56.1385,1902-12-25 -1902-05-05 12:34:56.1386,1902-12-25 -1902-05-05 12:34:56.1387,1902-12-25 -1902-05-05 12:34:56.1388,1902-12-25 -1902-05-05 12:34:56.1389,1902-12-25 -1902-05-05 12:34:56.139,1902-12-25 -1902-05-05 12:34:56.1391,1902-12-25 -1902-05-05 12:34:56.1392,1902-12-25 -1902-05-05 12:34:56.1393,1902-12-25 -1902-05-05 12:34:56.1394,1902-12-25 -1902-05-05 12:34:56.1395,1902-12-25 -1902-05-05 12:34:56.1396,1902-12-25 -1902-05-05 12:34:56.1397,1902-12-25 -1902-05-05 12:34:56.1398,1902-12-25 -1902-05-05 12:34:56.1399,1902-12-25 -1902-05-05 12:34:56.14,1902-12-25 -1902-05-05 12:34:56.1401,1902-12-25 -1902-05-05 12:34:56.1402,1902-12-25 -1902-05-05 12:34:56.1403,1902-12-25 -1902-05-05 12:34:56.1404,1902-12-25 -1902-05-05 12:34:56.1405,1902-12-25 -1902-05-05 12:34:56.1406,1902-12-25 -1902-05-05 12:34:56.1407,1902-12-25 -1902-05-05 12:34:56.1408,1902-12-25 -1902-05-05 12:34:56.1409,1902-12-25 -1902-05-05 12:34:56.141,1902-12-25 -1902-05-05 12:34:56.1411,1902-12-25 -1902-05-05 12:34:56.1412,1902-12-25 -1902-05-05 12:34:56.1413,1902-12-25 -1902-05-05 12:34:56.1414,1902-12-25 -1902-05-05 12:34:56.1415,1902-12-25 -1902-05-05 12:34:56.1416,1902-12-25 -1902-05-05 12:34:56.1417,1902-12-25 -1902-05-05 12:34:56.1418,1902-12-25 -1902-05-05 12:34:56.1419,1902-12-25 -1902-05-05 12:34:56.142,1902-12-25 -1902-05-05 12:34:56.1421,1902-12-25 -1902-05-05 12:34:56.1422,1902-12-25 -1902-05-05 12:34:56.1423,1902-12-25 -1902-05-05 12:34:56.1424,1902-12-25 -1902-05-05 12:34:56.1425,1902-12-25 -1902-05-05 12:34:56.1426,1902-12-25 -1902-05-05 12:34:56.1427,1902-12-25 -1902-05-05 12:34:56.1428,1902-12-25 -1902-05-05 12:34:56.1429,1902-12-25 -1902-05-05 12:34:56.143,1902-12-25 -1902-05-05 12:34:56.1431,1902-12-25 -1902-05-05 12:34:56.1432,1902-12-25 -1902-05-05 12:34:56.1433,1902-12-25 -1902-05-05 12:34:56.1434,1902-12-25 -1902-05-05 12:34:56.1435,1902-12-25 -1902-05-05 12:34:56.1436,1902-12-25 -1902-05-05 12:34:56.1437,1902-12-25 -1902-05-05 12:34:56.1438,1902-12-25 -1902-05-05 12:34:56.1439,1902-12-25 -1902-05-05 12:34:56.144,1902-12-25 -1902-05-05 12:34:56.1441,1902-12-25 -1902-05-05 12:34:56.1442,1902-12-25 -1902-05-05 12:34:56.1443,1902-12-25 -1902-05-05 12:34:56.1444,1902-12-25 -1902-05-05 12:34:56.1445,1902-12-25 -1902-05-05 12:34:56.1446,1902-12-25 -1902-05-05 12:34:56.1447,1902-12-25 -1902-05-05 12:34:56.1448,1902-12-25 -1902-05-05 12:34:56.1449,1902-12-25 -1902-05-05 12:34:56.145,1902-12-25 -1902-05-05 12:34:56.1451,1902-12-25 -1902-05-05 12:34:56.1452,1902-12-25 -1902-05-05 12:34:56.1453,1902-12-25 -1902-05-05 12:34:56.1454,1902-12-25 -1902-05-05 12:34:56.1455,1902-12-25 -1902-05-05 12:34:56.1456,1902-12-25 -1902-05-05 12:34:56.1457,1902-12-25 -1902-05-05 12:34:56.1458,1902-12-25 -1902-05-05 12:34:56.1459,1902-12-25 -1902-05-05 12:34:56.146,1902-12-25 -1902-05-05 12:34:56.1461,1902-12-25 -1902-05-05 12:34:56.1462,1902-12-25 -1902-05-05 12:34:56.1463,1902-12-25 -1902-05-05 12:34:56.1464,1902-12-25 -1902-05-05 12:34:56.1465,1902-12-25 -1902-05-05 12:34:56.1466,1902-12-25 -1902-05-05 12:34:56.1467,1902-12-25 -1902-05-05 12:34:56.1468,1902-12-25 -1902-05-05 12:34:56.1469,1902-12-25 -1902-05-05 12:34:56.147,1902-12-25 -1902-05-05 12:34:56.1471,1902-12-25 -1902-05-05 12:34:56.1472,1902-12-25 -1902-05-05 12:34:56.1473,1902-12-25 -1902-05-05 12:34:56.1474,1902-12-25 -1902-05-05 12:34:56.1475,1902-12-25 -1902-05-05 12:34:56.1476,1902-12-25 -1902-05-05 12:34:56.1477,1902-12-25 -1902-05-05 12:34:56.1478,1902-12-25 -1902-05-05 12:34:56.1479,1902-12-25 -1902-05-05 12:34:56.148,1902-12-25 -1902-05-05 12:34:56.1481,1902-12-25 -1902-05-05 12:34:56.1482,1902-12-25 -1902-05-05 12:34:56.1483,1902-12-25 -1902-05-05 12:34:56.1484,1902-12-25 -1902-05-05 12:34:56.1485,1902-12-25 -1902-05-05 12:34:56.1486,1902-12-25 -1902-05-05 12:34:56.1487,1902-12-25 -1902-05-05 12:34:56.1488,1902-12-25 -1902-05-05 12:34:56.1489,1902-12-25 -1902-05-05 12:34:56.149,1902-12-25 -1902-05-05 12:34:56.1491,1902-12-25 -1902-05-05 12:34:56.1492,1902-12-25 -1902-05-05 12:34:56.1493,1902-12-25 -1902-05-05 12:34:56.1494,1902-12-25 -1902-05-05 12:34:56.1495,1902-12-25 -1902-05-05 12:34:56.1496,1902-12-25 -1902-05-05 12:34:56.1497,1902-12-25 -1902-05-05 12:34:56.1498,1902-12-25 -1902-05-05 12:34:56.1499,1902-12-25 -1902-05-05 12:34:56.15,1902-12-25 -1902-05-05 12:34:56.1501,1902-12-25 -1902-05-05 12:34:56.1502,1902-12-25 -1902-05-05 12:34:56.1503,1902-12-25 -1902-05-05 12:34:56.1504,1902-12-25 -1902-05-05 12:34:56.1505,1902-12-25 -1902-05-05 12:34:56.1506,1902-12-25 -1902-05-05 12:34:56.1507,1902-12-25 -1902-05-05 12:34:56.1508,1902-12-25 -1902-05-05 12:34:56.1509,1902-12-25 -1902-05-05 12:34:56.151,1902-12-25 -1902-05-05 12:34:56.1511,1902-12-25 -1902-05-05 12:34:56.1512,1902-12-25 -1902-05-05 12:34:56.1513,1902-12-25 -1902-05-05 12:34:56.1514,1902-12-25 -1902-05-05 12:34:56.1515,1902-12-25 -1902-05-05 12:34:56.1516,1902-12-25 -1902-05-05 12:34:56.1517,1902-12-25 -1902-05-05 12:34:56.1518,1902-12-25 -1902-05-05 12:34:56.1519,1902-12-25 -1902-05-05 12:34:56.152,1902-12-25 -1902-05-05 12:34:56.1521,1902-12-25 -1902-05-05 12:34:56.1522,1902-12-25 -1902-05-05 12:34:56.1523,1902-12-25 -1902-05-05 12:34:56.1524,1902-12-25 -1902-05-05 12:34:56.1525,1902-12-25 -1902-05-05 12:34:56.1526,1902-12-25 -1902-05-05 12:34:56.1527,1902-12-25 -1902-05-05 12:34:56.1528,1902-12-25 -1902-05-05 12:34:56.1529,1902-12-25 -1902-05-05 12:34:56.153,1902-12-25 -1902-05-05 12:34:56.1531,1902-12-25 -1902-05-05 12:34:56.1532,1902-12-25 -1902-05-05 12:34:56.1533,1902-12-25 -1902-05-05 12:34:56.1534,1902-12-25 -1902-05-05 12:34:56.1535,1902-12-25 -1902-05-05 12:34:56.1536,1902-12-25 -1902-05-05 12:34:56.1537,1902-12-25 -1902-05-05 12:34:56.1538,1902-12-25 -1902-05-05 12:34:56.1539,1902-12-25 -1902-05-05 12:34:56.154,1902-12-25 -1902-05-05 12:34:56.1541,1902-12-25 -1902-05-05 12:34:56.1542,1902-12-25 -1902-05-05 12:34:56.1543,1902-12-25 -1902-05-05 12:34:56.1544,1902-12-25 -1902-05-05 12:34:56.1545,1902-12-25 -1902-05-05 12:34:56.1546,1902-12-25 -1902-05-05 12:34:56.1547,1902-12-25 -1902-05-05 12:34:56.1548,1902-12-25 -1902-05-05 12:34:56.1549,1902-12-25 -1902-05-05 12:34:56.155,1902-12-25 -1902-05-05 12:34:56.1551,1902-12-25 -1902-05-05 12:34:56.1552,1902-12-25 -1902-05-05 12:34:56.1553,1902-12-25 -1902-05-05 12:34:56.1554,1902-12-25 -1902-05-05 12:34:56.1555,1902-12-25 -1902-05-05 12:34:56.1556,1902-12-25 -1902-05-05 12:34:56.1557,1902-12-25 -1902-05-05 12:34:56.1558,1902-12-25 -1902-05-05 12:34:56.1559,1902-12-25 -1902-05-05 12:34:56.156,1902-12-25 -1902-05-05 12:34:56.1561,1902-12-25 -1902-05-05 12:34:56.1562,1902-12-25 -1902-05-05 12:34:56.1563,1902-12-25 -1902-05-05 12:34:56.1564,1902-12-25 -1902-05-05 12:34:56.1565,1902-12-25 -1902-05-05 12:34:56.1566,1902-12-25 -1902-05-05 12:34:56.1567,1902-12-25 -1902-05-05 12:34:56.1568,1902-12-25 -1902-05-05 12:34:56.1569,1902-12-25 -1902-05-05 12:34:56.157,1902-12-25 -1902-05-05 12:34:56.1571,1902-12-25 -1902-05-05 12:34:56.1572,1902-12-25 -1902-05-05 12:34:56.1573,1902-12-25 -1902-05-05 12:34:56.1574,1902-12-25 -1902-05-05 12:34:56.1575,1902-12-25 -1902-05-05 12:34:56.1576,1902-12-25 -1902-05-05 12:34:56.1577,1902-12-25 -1902-05-05 12:34:56.1578,1902-12-25 -1902-05-05 12:34:56.1579,1902-12-25 -1902-05-05 12:34:56.158,1902-12-25 -1902-05-05 12:34:56.1581,1902-12-25 -1902-05-05 12:34:56.1582,1902-12-25 -1902-05-05 12:34:56.1583,1902-12-25 -1902-05-05 12:34:56.1584,1902-12-25 -1902-05-05 12:34:56.1585,1902-12-25 -1902-05-05 12:34:56.1586,1902-12-25 -1902-05-05 12:34:56.1587,1902-12-25 -1902-05-05 12:34:56.1588,1902-12-25 -1902-05-05 12:34:56.1589,1902-12-25 -1902-05-05 12:34:56.159,1902-12-25 -1902-05-05 12:34:56.1591,1902-12-25 -1902-05-05 12:34:56.1592,1902-12-25 -1902-05-05 12:34:56.1593,1902-12-25 -1902-05-05 12:34:56.1594,1902-12-25 -1902-05-05 12:34:56.1595,1902-12-25 -1902-05-05 12:34:56.1596,1902-12-25 -1902-05-05 12:34:56.1597,1902-12-25 -1902-05-05 12:34:56.1598,1902-12-25 -1902-05-05 12:34:56.1599,1902-12-25 -1902-05-05 12:34:56.16,1902-12-25 -1902-05-05 12:34:56.1601,1902-12-25 -1902-05-05 12:34:56.1602,1902-12-25 -1902-05-05 12:34:56.1603,1902-12-25 -1902-05-05 12:34:56.1604,1902-12-25 -1902-05-05 12:34:56.1605,1902-12-25 -1902-05-05 12:34:56.1606,1902-12-25 -1902-05-05 12:34:56.1607,1902-12-25 -1902-05-05 12:34:56.1608,1902-12-25 -1902-05-05 12:34:56.1609,1902-12-25 -1902-05-05 12:34:56.161,1902-12-25 -1902-05-05 12:34:56.1611,1902-12-25 -1902-05-05 12:34:56.1612,1902-12-25 -1902-05-05 12:34:56.1613,1902-12-25 -1902-05-05 12:34:56.1614,1902-12-25 -1902-05-05 12:34:56.1615,1902-12-25 -1902-05-05 12:34:56.1616,1902-12-25 -1902-05-05 12:34:56.1617,1902-12-25 -1902-05-05 12:34:56.1618,1902-12-25 -1902-05-05 12:34:56.1619,1902-12-25 -1902-05-05 12:34:56.162,1902-12-25 -1902-05-05 12:34:56.1621,1902-12-25 -1902-05-05 12:34:56.1622,1902-12-25 -1902-05-05 12:34:56.1623,1902-12-25 -1902-05-05 12:34:56.1624,1902-12-25 -1902-05-05 12:34:56.1625,1902-12-25 -1902-05-05 12:34:56.1626,1902-12-25 -1902-05-05 12:34:56.1627,1902-12-25 -1902-05-05 12:34:56.1628,1902-12-25 -1902-05-05 12:34:56.1629,1902-12-25 -1902-05-05 12:34:56.163,1902-12-25 -1902-05-05 12:34:56.1631,1902-12-25 -1902-05-05 12:34:56.1632,1902-12-25 -1902-05-05 12:34:56.1633,1902-12-25 -1902-05-05 12:34:56.1634,1902-12-25 -1902-05-05 12:34:56.1635,1902-12-25 -1902-05-05 12:34:56.1636,1902-12-25 -1902-05-05 12:34:56.1637,1902-12-25 -1902-05-05 12:34:56.1638,1902-12-25 -1902-05-05 12:34:56.1639,1902-12-25 -1902-05-05 12:34:56.164,1902-12-25 -1902-05-05 12:34:56.1641,1902-12-25 -1902-05-05 12:34:56.1642,1902-12-25 -1902-05-05 12:34:56.1643,1902-12-25 -1902-05-05 12:34:56.1644,1902-12-25 -1902-05-05 12:34:56.1645,1902-12-25 -1902-05-05 12:34:56.1646,1902-12-25 -1902-05-05 12:34:56.1647,1902-12-25 -1902-05-05 12:34:56.1648,1902-12-25 -1902-05-05 12:34:56.1649,1902-12-25 -1902-05-05 12:34:56.165,1902-12-25 -1902-05-05 12:34:56.1651,1902-12-25 -1902-05-05 12:34:56.1652,1902-12-25 -1902-05-05 12:34:56.1653,1902-12-25 -1902-05-05 12:34:56.1654,1902-12-25 -1902-05-05 12:34:56.1655,1902-12-25 -1902-05-05 12:34:56.1656,1902-12-25 -1902-05-05 12:34:56.1657,1902-12-25 -1902-05-05 12:34:56.1658,1902-12-25 -1902-05-05 12:34:56.1659,1902-12-25 -1902-05-05 12:34:56.166,1902-12-25 -1902-05-05 12:34:56.1661,1902-12-25 -1902-05-05 12:34:56.1662,1902-12-25 -1902-05-05 12:34:56.1663,1902-12-25 -1902-05-05 12:34:56.1664,1902-12-25 -1902-05-05 12:34:56.1665,1902-12-25 -1902-05-05 12:34:56.1666,1902-12-25 -1902-05-05 12:34:56.1667,1902-12-25 -1902-05-05 12:34:56.1668,1902-12-25 -1902-05-05 12:34:56.1669,1902-12-25 -1902-05-05 12:34:56.167,1902-12-25 -1902-05-05 12:34:56.1671,1902-12-25 -1902-05-05 12:34:56.1672,1902-12-25 -1902-05-05 12:34:56.1673,1902-12-25 -1902-05-05 12:34:56.1674,1902-12-25 -1902-05-05 12:34:56.1675,1902-12-25 -1902-05-05 12:34:56.1676,1902-12-25 -1902-05-05 12:34:56.1677,1902-12-25 -1902-05-05 12:34:56.1678,1902-12-25 -1902-05-05 12:34:56.1679,1902-12-25 -1902-05-05 12:34:56.168,1902-12-25 -1902-05-05 12:34:56.1681,1902-12-25 -1902-05-05 12:34:56.1682,1902-12-25 -1902-05-05 12:34:56.1683,1902-12-25 -1902-05-05 12:34:56.1684,1902-12-25 -1902-05-05 12:34:56.1685,1902-12-25 -1902-05-05 12:34:56.1686,1902-12-25 -1902-05-05 12:34:56.1687,1902-12-25 -1902-05-05 12:34:56.1688,1902-12-25 -1902-05-05 12:34:56.1689,1902-12-25 -1902-05-05 12:34:56.169,1902-12-25 -1902-05-05 12:34:56.1691,1902-12-25 -1902-05-05 12:34:56.1692,1902-12-25 -1902-05-05 12:34:56.1693,1902-12-25 -1902-05-05 12:34:56.1694,1902-12-25 -1902-05-05 12:34:56.1695,1902-12-25 -1902-05-05 12:34:56.1696,1902-12-25 -1902-05-05 12:34:56.1697,1902-12-25 -1902-05-05 12:34:56.1698,1902-12-25 -1902-05-05 12:34:56.1699,1902-12-25 -1902-05-05 12:34:56.17,1902-12-25 -1902-05-05 12:34:56.1701,1902-12-25 -1902-05-05 12:34:56.1702,1902-12-25 -1902-05-05 12:34:56.1703,1902-12-25 -1902-05-05 12:34:56.1704,1902-12-25 -1902-05-05 12:34:56.1705,1902-12-25 -1902-05-05 12:34:56.1706,1902-12-25 -1902-05-05 12:34:56.1707,1902-12-25 -1902-05-05 12:34:56.1708,1902-12-25 -1902-05-05 12:34:56.1709,1902-12-25 -1902-05-05 12:34:56.171,1902-12-25 -1902-05-05 12:34:56.1711,1902-12-25 -1902-05-05 12:34:56.1712,1902-12-25 -1902-05-05 12:34:56.1713,1902-12-25 -1902-05-05 12:34:56.1714,1902-12-25 -1902-05-05 12:34:56.1715,1902-12-25 -1902-05-05 12:34:56.1716,1902-12-25 -1902-05-05 12:34:56.1717,1902-12-25 -1902-05-05 12:34:56.1718,1902-12-25 -1902-05-05 12:34:56.1719,1902-12-25 -1902-05-05 12:34:56.172,1902-12-25 -1902-05-05 12:34:56.1721,1902-12-25 -1902-05-05 12:34:56.1722,1902-12-25 -1902-05-05 12:34:56.1723,1902-12-25 -1902-05-05 12:34:56.1724,1902-12-25 -1902-05-05 12:34:56.1725,1902-12-25 -1902-05-05 12:34:56.1726,1902-12-25 -1902-05-05 12:34:56.1727,1902-12-25 -1902-05-05 12:34:56.1728,1902-12-25 -1902-05-05 12:34:56.1729,1902-12-25 -1902-05-05 12:34:56.173,1902-12-25 -1902-05-05 12:34:56.1731,1902-12-25 -1902-05-05 12:34:56.1732,1902-12-25 -1902-05-05 12:34:56.1733,1902-12-25 -1902-05-05 12:34:56.1734,1902-12-25 -1902-05-05 12:34:56.1735,1902-12-25 -1902-05-05 12:34:56.1736,1902-12-25 -1902-05-05 12:34:56.1737,1902-12-25 -1902-05-05 12:34:56.1738,1902-12-25 -1902-05-05 12:34:56.1739,1902-12-25 -1902-05-05 12:34:56.174,1902-12-25 -1902-05-05 12:34:56.1741,1902-12-25 -1902-05-05 12:34:56.1742,1902-12-25 -1902-05-05 12:34:56.1743,1902-12-25 -1902-05-05 12:34:56.1744,1902-12-25 -1902-05-05 12:34:56.1745,1902-12-25 -1902-05-05 12:34:56.1746,1902-12-25 -1902-05-05 12:34:56.1747,1902-12-25 -1902-05-05 12:34:56.1748,1902-12-25 -1902-05-05 12:34:56.1749,1902-12-25 -1902-05-05 12:34:56.175,1902-12-25 -1902-05-05 12:34:56.1751,1902-12-25 -1902-05-05 12:34:56.1752,1902-12-25 -1902-05-05 12:34:56.1753,1902-12-25 -1902-05-05 12:34:56.1754,1902-12-25 -1902-05-05 12:34:56.1755,1902-12-25 -1902-05-05 12:34:56.1756,1902-12-25 -1902-05-05 12:34:56.1757,1902-12-25 -1902-05-05 12:34:56.1758,1902-12-25 -1902-05-05 12:34:56.1759,1902-12-25 -1902-05-05 12:34:56.176,1902-12-25 -1902-05-05 12:34:56.1761,1902-12-25 -1902-05-05 12:34:56.1762,1902-12-25 -1902-05-05 12:34:56.1763,1902-12-25 -1902-05-05 12:34:56.1764,1902-12-25 -1902-05-05 12:34:56.1765,1902-12-25 -1902-05-05 12:34:56.1766,1902-12-25 -1902-05-05 12:34:56.1767,1902-12-25 -1902-05-05 12:34:56.1768,1902-12-25 -1902-05-05 12:34:56.1769,1902-12-25 -1902-05-05 12:34:56.177,1902-12-25 -1902-05-05 12:34:56.1771,1902-12-25 -1902-05-05 12:34:56.1772,1902-12-25 -1902-05-05 12:34:56.1773,1902-12-25 -1902-05-05 12:34:56.1774,1902-12-25 -1902-05-05 12:34:56.1775,1902-12-25 -1902-05-05 12:34:56.1776,1902-12-25 -1902-05-05 12:34:56.1777,1902-12-25 -1902-05-05 12:34:56.1778,1902-12-25 -1902-05-05 12:34:56.1779,1902-12-25 -1902-05-05 12:34:56.178,1902-12-25 -1902-05-05 12:34:56.1781,1902-12-25 -1902-05-05 12:34:56.1782,1902-12-25 -1902-05-05 12:34:56.1783,1902-12-25 -1902-05-05 12:34:56.1784,1902-12-25 -1902-05-05 12:34:56.1785,1902-12-25 -1902-05-05 12:34:56.1786,1902-12-25 -1902-05-05 12:34:56.1787,1902-12-25 -1902-05-05 12:34:56.1788,1902-12-25 -1902-05-05 12:34:56.1789,1902-12-25 -1902-05-05 12:34:56.179,1902-12-25 -1902-05-05 12:34:56.1791,1902-12-25 -1902-05-05 12:34:56.1792,1902-12-25 -1902-05-05 12:34:56.1793,1902-12-25 -1902-05-05 12:34:56.1794,1902-12-25 -1902-05-05 12:34:56.1795,1902-12-25 -1902-05-05 12:34:56.1796,1902-12-25 -1902-05-05 12:34:56.1797,1902-12-25 -1902-05-05 12:34:56.1798,1902-12-25 -1902-05-05 12:34:56.1799,1902-12-25 -1902-05-05 12:34:56.18,1902-12-25 -1902-05-05 12:34:56.1801,1902-12-25 -1902-05-05 12:34:56.1802,1902-12-25 -1902-05-05 12:34:56.1803,1902-12-25 -1902-05-05 12:34:56.1804,1902-12-25 -1902-05-05 12:34:56.1805,1902-12-25 -1902-05-05 12:34:56.1806,1902-12-25 -1902-05-05 12:34:56.1807,1902-12-25 -1902-05-05 12:34:56.1808,1902-12-25 -1902-05-05 12:34:56.1809,1902-12-25 -1902-05-05 12:34:56.181,1902-12-25 -1902-05-05 12:34:56.1811,1902-12-25 -1902-05-05 12:34:56.1812,1902-12-25 -1902-05-05 12:34:56.1813,1902-12-25 -1902-05-05 12:34:56.1814,1902-12-25 -1902-05-05 12:34:56.1815,1902-12-25 -1902-05-05 12:34:56.1816,1902-12-25 -1902-05-05 12:34:56.1817,1902-12-25 -1902-05-05 12:34:56.1818,1902-12-25 -1902-05-05 12:34:56.1819,1902-12-25 -1902-05-05 12:34:56.182,1902-12-25 -1902-05-05 12:34:56.1821,1902-12-25 -1902-05-05 12:34:56.1822,1902-12-25 -1902-05-05 12:34:56.1823,1902-12-25 -1902-05-05 12:34:56.1824,1902-12-25 -1902-05-05 12:34:56.1825,1902-12-25 -1902-05-05 12:34:56.1826,1902-12-25 -1902-05-05 12:34:56.1827,1902-12-25 -1902-05-05 12:34:56.1828,1902-12-25 -1902-05-05 12:34:56.1829,1902-12-25 -1902-05-05 12:34:56.183,1902-12-25 -1902-05-05 12:34:56.1831,1902-12-25 -1902-05-05 12:34:56.1832,1902-12-25 -1902-05-05 12:34:56.1833,1902-12-25 -1902-05-05 12:34:56.1834,1902-12-25 -1902-05-05 12:34:56.1835,1902-12-25 -1902-05-05 12:34:56.1836,1902-12-25 -1902-05-05 12:34:56.1837,1902-12-25 -1902-05-05 12:34:56.1838,1902-12-25 -1902-05-05 12:34:56.1839,1902-12-25 -1902-05-05 12:34:56.184,1902-12-25 -1902-05-05 12:34:56.1841,1902-12-25 -1902-05-05 12:34:56.1842,1902-12-25 -1902-05-05 12:34:56.1843,1902-12-25 -1902-05-05 12:34:56.1844,1902-12-25 -1902-05-05 12:34:56.1845,1902-12-25 -1902-05-05 12:34:56.1846,1902-12-25 -1902-05-05 12:34:56.1847,1902-12-25 -1902-05-05 12:34:56.1848,1902-12-25 -1902-05-05 12:34:56.1849,1902-12-25 -1902-05-05 12:34:56.185,1902-12-25 -1902-05-05 12:34:56.1851,1902-12-25 -1902-05-05 12:34:56.1852,1902-12-25 -1902-05-05 12:34:56.1853,1902-12-25 -1902-05-05 12:34:56.1854,1902-12-25 -1902-05-05 12:34:56.1855,1902-12-25 -1902-05-05 12:34:56.1856,1902-12-25 -1902-05-05 12:34:56.1857,1902-12-25 -1902-05-05 12:34:56.1858,1902-12-25 -1902-05-05 12:34:56.1859,1902-12-25 -1902-05-05 12:34:56.186,1902-12-25 -1902-05-05 12:34:56.1861,1902-12-25 -1902-05-05 12:34:56.1862,1902-12-25 -1902-05-05 12:34:56.1863,1902-12-25 -1902-05-05 12:34:56.1864,1902-12-25 -1902-05-05 12:34:56.1865,1902-12-25 -1902-05-05 12:34:56.1866,1902-12-25 -1902-05-05 12:34:56.1867,1902-12-25 -1902-05-05 12:34:56.1868,1902-12-25 -1902-05-05 12:34:56.1869,1902-12-25 -1902-05-05 12:34:56.187,1902-12-25 -1902-05-05 12:34:56.1871,1902-12-25 -1902-05-05 12:34:56.1872,1902-12-25 -1902-05-05 12:34:56.1873,1902-12-25 -1902-05-05 12:34:56.1874,1902-12-25 -1902-05-05 12:34:56.1875,1902-12-25 -1902-05-05 12:34:56.1876,1902-12-25 -1902-05-05 12:34:56.1877,1902-12-25 -1902-05-05 12:34:56.1878,1902-12-25 -1902-05-05 12:34:56.1879,1902-12-25 -1902-05-05 12:34:56.188,1902-12-25 -1902-05-05 12:34:56.1881,1902-12-25 -1902-05-05 12:34:56.1882,1902-12-25 -1902-05-05 12:34:56.1883,1902-12-25 -1902-05-05 12:34:56.1884,1902-12-25 -1902-05-05 12:34:56.1885,1902-12-25 -1902-05-05 12:34:56.1886,1902-12-25 -1902-05-05 12:34:56.1887,1902-12-25 -1902-05-05 12:34:56.1888,1902-12-25 -1902-05-05 12:34:56.1889,1902-12-25 -1902-05-05 12:34:56.189,1902-12-25 -1902-05-05 12:34:56.1891,1902-12-25 -1902-05-05 12:34:56.1892,1902-12-25 -1902-05-05 12:34:56.1893,1902-12-25 -1902-05-05 12:34:56.1894,1902-12-25 -1902-05-05 12:34:56.1895,1902-12-25 -1902-05-05 12:34:56.1896,1902-12-25 -1902-05-05 12:34:56.1897,1902-12-25 -1902-05-05 12:34:56.1898,1902-12-25 -1902-05-05 12:34:56.1899,1902-12-25 -1902-05-05 12:34:56.19,1902-12-25 -1902-05-05 12:34:56.1901,1902-12-25 -1902-05-05 12:34:56.1902,1902-12-25 -1902-05-05 12:34:56.1903,1902-12-25 -1902-05-05 12:34:56.1904,1902-12-25 -1902-05-05 12:34:56.1905,1902-12-25 -1902-05-05 12:34:56.1906,1902-12-25 -1902-05-05 12:34:56.1907,1902-12-25 -1902-05-05 12:34:56.1908,1902-12-25 -1902-05-05 12:34:56.1909,1902-12-25 -1902-05-05 12:34:56.191,1902-12-25 -1902-05-05 12:34:56.1911,1902-12-25 -1902-05-05 12:34:56.1912,1902-12-25 -1902-05-05 12:34:56.1913,1902-12-25 -1902-05-05 12:34:56.1914,1902-12-25 -1902-05-05 12:34:56.1915,1902-12-25 -1902-05-05 12:34:56.1916,1902-12-25 -1902-05-05 12:34:56.1917,1902-12-25 -1902-05-05 12:34:56.1918,1902-12-25 -1902-05-05 12:34:56.1919,1902-12-25 -1902-05-05 12:34:56.192,1902-12-25 -1902-05-05 12:34:56.1921,1902-12-25 -1902-05-05 12:34:56.1922,1902-12-25 -1902-05-05 12:34:56.1923,1902-12-25 -1902-05-05 12:34:56.1924,1902-12-25 -1902-05-05 12:34:56.1925,1902-12-25 -1902-05-05 12:34:56.1926,1902-12-25 -1902-05-05 12:34:56.1927,1902-12-25 -1902-05-05 12:34:56.1928,1902-12-25 -1902-05-05 12:34:56.1929,1902-12-25 -1902-05-05 12:34:56.193,1902-12-25 -1902-05-05 12:34:56.1931,1902-12-25 -1902-05-05 12:34:56.1932,1902-12-25 -1902-05-05 12:34:56.1933,1902-12-25 -1902-05-05 12:34:56.1934,1902-12-25 -1902-05-05 12:34:56.1935,1902-12-25 -1902-05-05 12:34:56.1936,1902-12-25 -1902-05-05 12:34:56.1937,1902-12-25 -1902-05-05 12:34:56.1938,1902-12-25 -1902-05-05 12:34:56.1939,1902-12-25 -1902-05-05 12:34:56.194,1902-12-25 -1902-05-05 12:34:56.1941,1902-12-25 -1902-05-05 12:34:56.1942,1902-12-25 -1902-05-05 12:34:56.1943,1902-12-25 -1902-05-05 12:34:56.1944,1902-12-25 -1902-05-05 12:34:56.1945,1902-12-25 -1902-05-05 12:34:56.1946,1902-12-25 -1902-05-05 12:34:56.1947,1902-12-25 -1902-05-05 12:34:56.1948,1902-12-25 -1902-05-05 12:34:56.1949,1902-12-25 -1902-05-05 12:34:56.195,1902-12-25 -1902-05-05 12:34:56.1951,1902-12-25 -1902-05-05 12:34:56.1952,1902-12-25 -1902-05-05 12:34:56.1953,1902-12-25 -1902-05-05 12:34:56.1954,1902-12-25 -1902-05-05 12:34:56.1955,1902-12-25 -1902-05-05 12:34:56.1956,1902-12-25 -1902-05-05 12:34:56.1957,1902-12-25 -1902-05-05 12:34:56.1958,1902-12-25 -1902-05-05 12:34:56.1959,1902-12-25 -1902-05-05 12:34:56.196,1902-12-25 -1902-05-05 12:34:56.1961,1902-12-25 -1902-05-05 12:34:56.1962,1902-12-25 -1902-05-05 12:34:56.1963,1902-12-25 -1902-05-05 12:34:56.1964,1902-12-25 -1902-05-05 12:34:56.1965,1902-12-25 -1902-05-05 12:34:56.1966,1902-12-25 -1902-05-05 12:34:56.1967,1902-12-25 -1902-05-05 12:34:56.1968,1902-12-25 -1902-05-05 12:34:56.1969,1902-12-25 -1902-05-05 12:34:56.197,1902-12-25 -1902-05-05 12:34:56.1971,1902-12-25 -1902-05-05 12:34:56.1972,1902-12-25 -1902-05-05 12:34:56.1973,1902-12-25 -1902-05-05 12:34:56.1974,1902-12-25 -1902-05-05 12:34:56.1975,1902-12-25 -1902-05-05 12:34:56.1976,1902-12-25 -1902-05-05 12:34:56.1977,1902-12-25 -1902-05-05 12:34:56.1978,1902-12-25 -1902-05-05 12:34:56.1979,1902-12-25 -1902-05-05 12:34:56.198,1902-12-25 -1902-05-05 12:34:56.1981,1902-12-25 -1902-05-05 12:34:56.1982,1902-12-25 -1902-05-05 12:34:56.1983,1902-12-25 -1902-05-05 12:34:56.1984,1902-12-25 -1902-05-05 12:34:56.1985,1902-12-25 -1902-05-05 12:34:56.1986,1902-12-25 -1902-05-05 12:34:56.1987,1902-12-25 -1902-05-05 12:34:56.1988,1902-12-25 -1902-05-05 12:34:56.1989,1902-12-25 -1902-05-05 12:34:56.199,1902-12-25 -1902-05-05 12:34:56.1991,1902-12-25 -1902-05-05 12:34:56.1992,1902-12-25 -1902-05-05 12:34:56.1993,1902-12-25 -1902-05-05 12:34:56.1994,1902-12-25 -1902-05-05 12:34:56.1995,1902-12-25 -1902-05-05 12:34:56.1996,1902-12-25 -1902-05-05 12:34:56.1997,1902-12-25 -1902-05-05 12:34:56.1998,1902-12-25 -1902-05-05 12:34:56.1999,1902-12-25 -1903-05-05 12:34:56.1,1903-12-25 -1903-05-05 12:34:56.1001,1903-12-25 -1903-05-05 12:34:56.1002,1903-12-25 -1903-05-05 12:34:56.1003,1903-12-25 -1903-05-05 12:34:56.1004,1903-12-25 -1903-05-05 12:34:56.1005,1903-12-25 -1903-05-05 12:34:56.1006,1903-12-25 -1903-05-05 12:34:56.1007,1903-12-25 -1903-05-05 12:34:56.1008,1903-12-25 -1903-05-05 12:34:56.1009,1903-12-25 -1903-05-05 12:34:56.101,1903-12-25 -1903-05-05 12:34:56.1011,1903-12-25 -1903-05-05 12:34:56.1012,1903-12-25 -1903-05-05 12:34:56.1013,1903-12-25 -1903-05-05 12:34:56.1014,1903-12-25 -1903-05-05 12:34:56.1015,1903-12-25 -1903-05-05 12:34:56.1016,1903-12-25 -1903-05-05 12:34:56.1017,1903-12-25 -1903-05-05 12:34:56.1018,1903-12-25 -1903-05-05 12:34:56.1019,1903-12-25 -1903-05-05 12:34:56.102,1903-12-25 -1903-05-05 12:34:56.1021,1903-12-25 -1903-05-05 12:34:56.1022,1903-12-25 -1903-05-05 12:34:56.1023,1903-12-25 -1903-05-05 12:34:56.1024,1903-12-25 -1903-05-05 12:34:56.1025,1903-12-25 -1903-05-05 12:34:56.1026,1903-12-25 -1903-05-05 12:34:56.1027,1903-12-25 -1903-05-05 12:34:56.1028,1903-12-25 -1903-05-05 12:34:56.1029,1903-12-25 -1903-05-05 12:34:56.103,1903-12-25 -1903-05-05 12:34:56.1031,1903-12-25 -1903-05-05 12:34:56.1032,1903-12-25 -1903-05-05 12:34:56.1033,1903-12-25 -1903-05-05 12:34:56.1034,1903-12-25 -1903-05-05 12:34:56.1035,1903-12-25 -1903-05-05 12:34:56.1036,1903-12-25 -1903-05-05 12:34:56.1037,1903-12-25 -1903-05-05 12:34:56.1038,1903-12-25 -1903-05-05 12:34:56.1039,1903-12-25 -1903-05-05 12:34:56.104,1903-12-25 -1903-05-05 12:34:56.1041,1903-12-25 -1903-05-05 12:34:56.1042,1903-12-25 -1903-05-05 12:34:56.1043,1903-12-25 -1903-05-05 12:34:56.1044,1903-12-25 -1903-05-05 12:34:56.1045,1903-12-25 -1903-05-05 12:34:56.1046,1903-12-25 -1903-05-05 12:34:56.1047,1903-12-25 -1903-05-05 12:34:56.1048,1903-12-25 -1903-05-05 12:34:56.1049,1903-12-25 -1903-05-05 12:34:56.105,1903-12-25 -1903-05-05 12:34:56.1051,1903-12-25 -1903-05-05 12:34:56.1052,1903-12-25 -1903-05-05 12:34:56.1053,1903-12-25 -1903-05-05 12:34:56.1054,1903-12-25 -1903-05-05 12:34:56.1055,1903-12-25 -1903-05-05 12:34:56.1056,1903-12-25 -1903-05-05 12:34:56.1057,1903-12-25 -1903-05-05 12:34:56.1058,1903-12-25 -1903-05-05 12:34:56.1059,1903-12-25 -1903-05-05 12:34:56.106,1903-12-25 -1903-05-05 12:34:56.1061,1903-12-25 -1903-05-05 12:34:56.1062,1903-12-25 -1903-05-05 12:34:56.1063,1903-12-25 -1903-05-05 12:34:56.1064,1903-12-25 -1903-05-05 12:34:56.1065,1903-12-25 -1903-05-05 12:34:56.1066,1903-12-25 -1903-05-05 12:34:56.1067,1903-12-25 -1903-05-05 12:34:56.1068,1903-12-25 -1903-05-05 12:34:56.1069,1903-12-25 -1903-05-05 12:34:56.107,1903-12-25 -1903-05-05 12:34:56.1071,1903-12-25 -1903-05-05 12:34:56.1072,1903-12-25 -1903-05-05 12:34:56.1073,1903-12-25 -1903-05-05 12:34:56.1074,1903-12-25 -1903-05-05 12:34:56.1075,1903-12-25 -1903-05-05 12:34:56.1076,1903-12-25 -1903-05-05 12:34:56.1077,1903-12-25 -1903-05-05 12:34:56.1078,1903-12-25 -1903-05-05 12:34:56.1079,1903-12-25 -1903-05-05 12:34:56.108,1903-12-25 -1903-05-05 12:34:56.1081,1903-12-25 -1903-05-05 12:34:56.1082,1903-12-25 -1903-05-05 12:34:56.1083,1903-12-25 -1903-05-05 12:34:56.1084,1903-12-25 -1903-05-05 12:34:56.1085,1903-12-25 -1903-05-05 12:34:56.1086,1903-12-25 -1903-05-05 12:34:56.1087,1903-12-25 -1903-05-05 12:34:56.1088,1903-12-25 -1903-05-05 12:34:56.1089,1903-12-25 -1903-05-05 12:34:56.109,1903-12-25 -1903-05-05 12:34:56.1091,1903-12-25 -1903-05-05 12:34:56.1092,1903-12-25 -1903-05-05 12:34:56.1093,1903-12-25 -1903-05-05 12:34:56.1094,1903-12-25 -1903-05-05 12:34:56.1095,1903-12-25 -1903-05-05 12:34:56.1096,1903-12-25 -1903-05-05 12:34:56.1097,1903-12-25 -1903-05-05 12:34:56.1098,1903-12-25 -1903-05-05 12:34:56.1099,1903-12-25 -1903-05-05 12:34:56.11,1903-12-25 -1903-05-05 12:34:56.1101,1903-12-25 -1903-05-05 12:34:56.1102,1903-12-25 -1903-05-05 12:34:56.1103,1903-12-25 -1903-05-05 12:34:56.1104,1903-12-25 -1903-05-05 12:34:56.1105,1903-12-25 -1903-05-05 12:34:56.1106,1903-12-25 -1903-05-05 12:34:56.1107,1903-12-25 -1903-05-05 12:34:56.1108,1903-12-25 -1903-05-05 12:34:56.1109,1903-12-25 -1903-05-05 12:34:56.111,1903-12-25 -1903-05-05 12:34:56.1111,1903-12-25 -1903-05-05 12:34:56.1112,1903-12-25 -1903-05-05 12:34:56.1113,1903-12-25 -1903-05-05 12:34:56.1114,1903-12-25 -1903-05-05 12:34:56.1115,1903-12-25 -1903-05-05 12:34:56.1116,1903-12-25 -1903-05-05 12:34:56.1117,1903-12-25 -1903-05-05 12:34:56.1118,1903-12-25 -1903-05-05 12:34:56.1119,1903-12-25 -1903-05-05 12:34:56.112,1903-12-25 -1903-05-05 12:34:56.1121,1903-12-25 -1903-05-05 12:34:56.1122,1903-12-25 -1903-05-05 12:34:56.1123,1903-12-25 -1903-05-05 12:34:56.1124,1903-12-25 -1903-05-05 12:34:56.1125,1903-12-25 -1903-05-05 12:34:56.1126,1903-12-25 -1903-05-05 12:34:56.1127,1903-12-25 -1903-05-05 12:34:56.1128,1903-12-25 -1903-05-05 12:34:56.1129,1903-12-25 -1903-05-05 12:34:56.113,1903-12-25 -1903-05-05 12:34:56.1131,1903-12-25 -1903-05-05 12:34:56.1132,1903-12-25 -1903-05-05 12:34:56.1133,1903-12-25 -1903-05-05 12:34:56.1134,1903-12-25 -1903-05-05 12:34:56.1135,1903-12-25 -1903-05-05 12:34:56.1136,1903-12-25 -1903-05-05 12:34:56.1137,1903-12-25 -1903-05-05 12:34:56.1138,1903-12-25 -1903-05-05 12:34:56.1139,1903-12-25 -1903-05-05 12:34:56.114,1903-12-25 -1903-05-05 12:34:56.1141,1903-12-25 -1903-05-05 12:34:56.1142,1903-12-25 -1903-05-05 12:34:56.1143,1903-12-25 -1903-05-05 12:34:56.1144,1903-12-25 -1903-05-05 12:34:56.1145,1903-12-25 -1903-05-05 12:34:56.1146,1903-12-25 -1903-05-05 12:34:56.1147,1903-12-25 -1903-05-05 12:34:56.1148,1903-12-25 -1903-05-05 12:34:56.1149,1903-12-25 -1903-05-05 12:34:56.115,1903-12-25 -1903-05-05 12:34:56.1151,1903-12-25 -1903-05-05 12:34:56.1152,1903-12-25 -1903-05-05 12:34:56.1153,1903-12-25 -1903-05-05 12:34:56.1154,1903-12-25 -1903-05-05 12:34:56.1155,1903-12-25 -1903-05-05 12:34:56.1156,1903-12-25 -1903-05-05 12:34:56.1157,1903-12-25 -1903-05-05 12:34:56.1158,1903-12-25 -1903-05-05 12:34:56.1159,1903-12-25 -1903-05-05 12:34:56.116,1903-12-25 -1903-05-05 12:34:56.1161,1903-12-25 -1903-05-05 12:34:56.1162,1903-12-25 -1903-05-05 12:34:56.1163,1903-12-25 -1903-05-05 12:34:56.1164,1903-12-25 -1903-05-05 12:34:56.1165,1903-12-25 -1903-05-05 12:34:56.1166,1903-12-25 -1903-05-05 12:34:56.1167,1903-12-25 -1903-05-05 12:34:56.1168,1903-12-25 -1903-05-05 12:34:56.1169,1903-12-25 -1903-05-05 12:34:56.117,1903-12-25 -1903-05-05 12:34:56.1171,1903-12-25 -1903-05-05 12:34:56.1172,1903-12-25 -1903-05-05 12:34:56.1173,1903-12-25 -1903-05-05 12:34:56.1174,1903-12-25 -1903-05-05 12:34:56.1175,1903-12-25 -1903-05-05 12:34:56.1176,1903-12-25 -1903-05-05 12:34:56.1177,1903-12-25 -1903-05-05 12:34:56.1178,1903-12-25 -1903-05-05 12:34:56.1179,1903-12-25 -1903-05-05 12:34:56.118,1903-12-25 -1903-05-05 12:34:56.1181,1903-12-25 -1903-05-05 12:34:56.1182,1903-12-25 -1903-05-05 12:34:56.1183,1903-12-25 -1903-05-05 12:34:56.1184,1903-12-25 -1903-05-05 12:34:56.1185,1903-12-25 -1903-05-05 12:34:56.1186,1903-12-25 -1903-05-05 12:34:56.1187,1903-12-25 -1903-05-05 12:34:56.1188,1903-12-25 -1903-05-05 12:34:56.1189,1903-12-25 -1903-05-05 12:34:56.119,1903-12-25 -1903-05-05 12:34:56.1191,1903-12-25 -1903-05-05 12:34:56.1192,1903-12-25 -1903-05-05 12:34:56.1193,1903-12-25 -1903-05-05 12:34:56.1194,1903-12-25 -1903-05-05 12:34:56.1195,1903-12-25 -1903-05-05 12:34:56.1196,1903-12-25 -1903-05-05 12:34:56.1197,1903-12-25 -1903-05-05 12:34:56.1198,1903-12-25 -1903-05-05 12:34:56.1199,1903-12-25 -1903-05-05 12:34:56.12,1903-12-25 -1903-05-05 12:34:56.1201,1903-12-25 -1903-05-05 12:34:56.1202,1903-12-25 -1903-05-05 12:34:56.1203,1903-12-25 -1903-05-05 12:34:56.1204,1903-12-25 -1903-05-05 12:34:56.1205,1903-12-25 -1903-05-05 12:34:56.1206,1903-12-25 -1903-05-05 12:34:56.1207,1903-12-25 -1903-05-05 12:34:56.1208,1903-12-25 -1903-05-05 12:34:56.1209,1903-12-25 -1903-05-05 12:34:56.121,1903-12-25 -1903-05-05 12:34:56.1211,1903-12-25 -1903-05-05 12:34:56.1212,1903-12-25 -1903-05-05 12:34:56.1213,1903-12-25 -1903-05-05 12:34:56.1214,1903-12-25 -1903-05-05 12:34:56.1215,1903-12-25 -1903-05-05 12:34:56.1216,1903-12-25 -1903-05-05 12:34:56.1217,1903-12-25 -1903-05-05 12:34:56.1218,1903-12-25 -1903-05-05 12:34:56.1219,1903-12-25 -1903-05-05 12:34:56.122,1903-12-25 -1903-05-05 12:34:56.1221,1903-12-25 -1903-05-05 12:34:56.1222,1903-12-25 -1903-05-05 12:34:56.1223,1903-12-25 -1903-05-05 12:34:56.1224,1903-12-25 -1903-05-05 12:34:56.1225,1903-12-25 -1903-05-05 12:34:56.1226,1903-12-25 -1903-05-05 12:34:56.1227,1903-12-25 -1903-05-05 12:34:56.1228,1903-12-25 -1903-05-05 12:34:56.1229,1903-12-25 -1903-05-05 12:34:56.123,1903-12-25 -1903-05-05 12:34:56.1231,1903-12-25 -1903-05-05 12:34:56.1232,1903-12-25 -1903-05-05 12:34:56.1233,1903-12-25 -1903-05-05 12:34:56.1234,1903-12-25 -1903-05-05 12:34:56.1235,1903-12-25 -1903-05-05 12:34:56.1236,1903-12-25 -1903-05-05 12:34:56.1237,1903-12-25 -1903-05-05 12:34:56.1238,1903-12-25 -1903-05-05 12:34:56.1239,1903-12-25 -1903-05-05 12:34:56.124,1903-12-25 -1903-05-05 12:34:56.1241,1903-12-25 -1903-05-05 12:34:56.1242,1903-12-25 -1903-05-05 12:34:56.1243,1903-12-25 -1903-05-05 12:34:56.1244,1903-12-25 -1903-05-05 12:34:56.1245,1903-12-25 -1903-05-05 12:34:56.1246,1903-12-25 -1903-05-05 12:34:56.1247,1903-12-25 -1903-05-05 12:34:56.1248,1903-12-25 -1903-05-05 12:34:56.1249,1903-12-25 -1903-05-05 12:34:56.125,1903-12-25 -1903-05-05 12:34:56.1251,1903-12-25 -1903-05-05 12:34:56.1252,1903-12-25 -1903-05-05 12:34:56.1253,1903-12-25 -1903-05-05 12:34:56.1254,1903-12-25 -1903-05-05 12:34:56.1255,1903-12-25 -1903-05-05 12:34:56.1256,1903-12-25 -1903-05-05 12:34:56.1257,1903-12-25 -1903-05-05 12:34:56.1258,1903-12-25 -1903-05-05 12:34:56.1259,1903-12-25 -1903-05-05 12:34:56.126,1903-12-25 -1903-05-05 12:34:56.1261,1903-12-25 -1903-05-05 12:34:56.1262,1903-12-25 -1903-05-05 12:34:56.1263,1903-12-25 -1903-05-05 12:34:56.1264,1903-12-25 -1903-05-05 12:34:56.1265,1903-12-25 -1903-05-05 12:34:56.1266,1903-12-25 -1903-05-05 12:34:56.1267,1903-12-25 -1903-05-05 12:34:56.1268,1903-12-25 -1903-05-05 12:34:56.1269,1903-12-25 -1903-05-05 12:34:56.127,1903-12-25 -1903-05-05 12:34:56.1271,1903-12-25 -1903-05-05 12:34:56.1272,1903-12-25 -1903-05-05 12:34:56.1273,1903-12-25 -1903-05-05 12:34:56.1274,1903-12-25 -1903-05-05 12:34:56.1275,1903-12-25 -1903-05-05 12:34:56.1276,1903-12-25 -1903-05-05 12:34:56.1277,1903-12-25 -1903-05-05 12:34:56.1278,1903-12-25 -1903-05-05 12:34:56.1279,1903-12-25 -1903-05-05 12:34:56.128,1903-12-25 -1903-05-05 12:34:56.1281,1903-12-25 -1903-05-05 12:34:56.1282,1903-12-25 -1903-05-05 12:34:56.1283,1903-12-25 -1903-05-05 12:34:56.1284,1903-12-25 -1903-05-05 12:34:56.1285,1903-12-25 -1903-05-05 12:34:56.1286,1903-12-25 -1903-05-05 12:34:56.1287,1903-12-25 -1903-05-05 12:34:56.1288,1903-12-25 -1903-05-05 12:34:56.1289,1903-12-25 -1903-05-05 12:34:56.129,1903-12-25 -1903-05-05 12:34:56.1291,1903-12-25 -1903-05-05 12:34:56.1292,1903-12-25 -1903-05-05 12:34:56.1293,1903-12-25 -1903-05-05 12:34:56.1294,1903-12-25 -1903-05-05 12:34:56.1295,1903-12-25 -1903-05-05 12:34:56.1296,1903-12-25 -1903-05-05 12:34:56.1297,1903-12-25 -1903-05-05 12:34:56.1298,1903-12-25 -1903-05-05 12:34:56.1299,1903-12-25 -1903-05-05 12:34:56.13,1903-12-25 -1903-05-05 12:34:56.1301,1903-12-25 -1903-05-05 12:34:56.1302,1903-12-25 -1903-05-05 12:34:56.1303,1903-12-25 -1903-05-05 12:34:56.1304,1903-12-25 -1903-05-05 12:34:56.1305,1903-12-25 -1903-05-05 12:34:56.1306,1903-12-25 -1903-05-05 12:34:56.1307,1903-12-25 -1903-05-05 12:34:56.1308,1903-12-25 -1903-05-05 12:34:56.1309,1903-12-25 -1903-05-05 12:34:56.131,1903-12-25 -1903-05-05 12:34:56.1311,1903-12-25 -1903-05-05 12:34:56.1312,1903-12-25 -1903-05-05 12:34:56.1313,1903-12-25 -1903-05-05 12:34:56.1314,1903-12-25 -1903-05-05 12:34:56.1315,1903-12-25 -1903-05-05 12:34:56.1316,1903-12-25 -1903-05-05 12:34:56.1317,1903-12-25 -1903-05-05 12:34:56.1318,1903-12-25 -1903-05-05 12:34:56.1319,1903-12-25 -1903-05-05 12:34:56.132,1903-12-25 -1903-05-05 12:34:56.1321,1903-12-25 -1903-05-05 12:34:56.1322,1903-12-25 -1903-05-05 12:34:56.1323,1903-12-25 -1903-05-05 12:34:56.1324,1903-12-25 -1903-05-05 12:34:56.1325,1903-12-25 -1903-05-05 12:34:56.1326,1903-12-25 -1903-05-05 12:34:56.1327,1903-12-25 -1903-05-05 12:34:56.1328,1903-12-25 -1903-05-05 12:34:56.1329,1903-12-25 -1903-05-05 12:34:56.133,1903-12-25 -1903-05-05 12:34:56.1331,1903-12-25 -1903-05-05 12:34:56.1332,1903-12-25 -1903-05-05 12:34:56.1333,1903-12-25 -1903-05-05 12:34:56.1334,1903-12-25 -1903-05-05 12:34:56.1335,1903-12-25 -1903-05-05 12:34:56.1336,1903-12-25 -1903-05-05 12:34:56.1337,1903-12-25 -1903-05-05 12:34:56.1338,1903-12-25 -1903-05-05 12:34:56.1339,1903-12-25 -1903-05-05 12:34:56.134,1903-12-25 -1903-05-05 12:34:56.1341,1903-12-25 -1903-05-05 12:34:56.1342,1903-12-25 -1903-05-05 12:34:56.1343,1903-12-25 -1903-05-05 12:34:56.1344,1903-12-25 -1903-05-05 12:34:56.1345,1903-12-25 -1903-05-05 12:34:56.1346,1903-12-25 -1903-05-05 12:34:56.1347,1903-12-25 -1903-05-05 12:34:56.1348,1903-12-25 -1903-05-05 12:34:56.1349,1903-12-25 -1903-05-05 12:34:56.135,1903-12-25 -1903-05-05 12:34:56.1351,1903-12-25 -1903-05-05 12:34:56.1352,1903-12-25 -1903-05-05 12:34:56.1353,1903-12-25 -1903-05-05 12:34:56.1354,1903-12-25 -1903-05-05 12:34:56.1355,1903-12-25 -1903-05-05 12:34:56.1356,1903-12-25 -1903-05-05 12:34:56.1357,1903-12-25 -1903-05-05 12:34:56.1358,1903-12-25 -1903-05-05 12:34:56.1359,1903-12-25 -1903-05-05 12:34:56.136,1903-12-25 -1903-05-05 12:34:56.1361,1903-12-25 -1903-05-05 12:34:56.1362,1903-12-25 -1903-05-05 12:34:56.1363,1903-12-25 -1903-05-05 12:34:56.1364,1903-12-25 -1903-05-05 12:34:56.1365,1903-12-25 -1903-05-05 12:34:56.1366,1903-12-25 -1903-05-05 12:34:56.1367,1903-12-25 -1903-05-05 12:34:56.1368,1903-12-25 -1903-05-05 12:34:56.1369,1903-12-25 -1903-05-05 12:34:56.137,1903-12-25 -1903-05-05 12:34:56.1371,1903-12-25 -1903-05-05 12:34:56.1372,1903-12-25 -1903-05-05 12:34:56.1373,1903-12-25 -1903-05-05 12:34:56.1374,1903-12-25 -1903-05-05 12:34:56.1375,1903-12-25 -1903-05-05 12:34:56.1376,1903-12-25 -1903-05-05 12:34:56.1377,1903-12-25 -1903-05-05 12:34:56.1378,1903-12-25 -1903-05-05 12:34:56.1379,1903-12-25 -1903-05-05 12:34:56.138,1903-12-25 -1903-05-05 12:34:56.1381,1903-12-25 -1903-05-05 12:34:56.1382,1903-12-25 -1903-05-05 12:34:56.1383,1903-12-25 -1903-05-05 12:34:56.1384,1903-12-25 -1903-05-05 12:34:56.1385,1903-12-25 -1903-05-05 12:34:56.1386,1903-12-25 -1903-05-05 12:34:56.1387,1903-12-25 -1903-05-05 12:34:56.1388,1903-12-25 -1903-05-05 12:34:56.1389,1903-12-25 -1903-05-05 12:34:56.139,1903-12-25 -1903-05-05 12:34:56.1391,1903-12-25 -1903-05-05 12:34:56.1392,1903-12-25 -1903-05-05 12:34:56.1393,1903-12-25 -1903-05-05 12:34:56.1394,1903-12-25 -1903-05-05 12:34:56.1395,1903-12-25 -1903-05-05 12:34:56.1396,1903-12-25 -1903-05-05 12:34:56.1397,1903-12-25 -1903-05-05 12:34:56.1398,1903-12-25 -1903-05-05 12:34:56.1399,1903-12-25 -1903-05-05 12:34:56.14,1903-12-25 -1903-05-05 12:34:56.1401,1903-12-25 -1903-05-05 12:34:56.1402,1903-12-25 -1903-05-05 12:34:56.1403,1903-12-25 -1903-05-05 12:34:56.1404,1903-12-25 -1903-05-05 12:34:56.1405,1903-12-25 -1903-05-05 12:34:56.1406,1903-12-25 -1903-05-05 12:34:56.1407,1903-12-25 -1903-05-05 12:34:56.1408,1903-12-25 -1903-05-05 12:34:56.1409,1903-12-25 -1903-05-05 12:34:56.141,1903-12-25 -1903-05-05 12:34:56.1411,1903-12-25 -1903-05-05 12:34:56.1412,1903-12-25 -1903-05-05 12:34:56.1413,1903-12-25 -1903-05-05 12:34:56.1414,1903-12-25 -1903-05-05 12:34:56.1415,1903-12-25 -1903-05-05 12:34:56.1416,1903-12-25 -1903-05-05 12:34:56.1417,1903-12-25 -1903-05-05 12:34:56.1418,1903-12-25 -1903-05-05 12:34:56.1419,1903-12-25 -1903-05-05 12:34:56.142,1903-12-25 -1903-05-05 12:34:56.1421,1903-12-25 -1903-05-05 12:34:56.1422,1903-12-25 -1903-05-05 12:34:56.1423,1903-12-25 -1903-05-05 12:34:56.1424,1903-12-25 -1903-05-05 12:34:56.1425,1903-12-25 -1903-05-05 12:34:56.1426,1903-12-25 -1903-05-05 12:34:56.1427,1903-12-25 -1903-05-05 12:34:56.1428,1903-12-25 -1903-05-05 12:34:56.1429,1903-12-25 -1903-05-05 12:34:56.143,1903-12-25 -1903-05-05 12:34:56.1431,1903-12-25 -1903-05-05 12:34:56.1432,1903-12-25 -1903-05-05 12:34:56.1433,1903-12-25 -1903-05-05 12:34:56.1434,1903-12-25 -1903-05-05 12:34:56.1435,1903-12-25 -1903-05-05 12:34:56.1436,1903-12-25 -1903-05-05 12:34:56.1437,1903-12-25 -1903-05-05 12:34:56.1438,1903-12-25 -1903-05-05 12:34:56.1439,1903-12-25 -1903-05-05 12:34:56.144,1903-12-25 -1903-05-05 12:34:56.1441,1903-12-25 -1903-05-05 12:34:56.1442,1903-12-25 -1903-05-05 12:34:56.1443,1903-12-25 -1903-05-05 12:34:56.1444,1903-12-25 -1903-05-05 12:34:56.1445,1903-12-25 -1903-05-05 12:34:56.1446,1903-12-25 -1903-05-05 12:34:56.1447,1903-12-25 -1903-05-05 12:34:56.1448,1903-12-25 -1903-05-05 12:34:56.1449,1903-12-25 -1903-05-05 12:34:56.145,1903-12-25 -1903-05-05 12:34:56.1451,1903-12-25 -1903-05-05 12:34:56.1452,1903-12-25 -1903-05-05 12:34:56.1453,1903-12-25 -1903-05-05 12:34:56.1454,1903-12-25 -1903-05-05 12:34:56.1455,1903-12-25 -1903-05-05 12:34:56.1456,1903-12-25 -1903-05-05 12:34:56.1457,1903-12-25 -1903-05-05 12:34:56.1458,1903-12-25 -1903-05-05 12:34:56.1459,1903-12-25 -1903-05-05 12:34:56.146,1903-12-25 -1903-05-05 12:34:56.1461,1903-12-25 -1903-05-05 12:34:56.1462,1903-12-25 -1903-05-05 12:34:56.1463,1903-12-25 -1903-05-05 12:34:56.1464,1903-12-25 -1903-05-05 12:34:56.1465,1903-12-25 -1903-05-05 12:34:56.1466,1903-12-25 -1903-05-05 12:34:56.1467,1903-12-25 -1903-05-05 12:34:56.1468,1903-12-25 -1903-05-05 12:34:56.1469,1903-12-25 -1903-05-05 12:34:56.147,1903-12-25 -1903-05-05 12:34:56.1471,1903-12-25 -1903-05-05 12:34:56.1472,1903-12-25 -1903-05-05 12:34:56.1473,1903-12-25 -1903-05-05 12:34:56.1474,1903-12-25 -1903-05-05 12:34:56.1475,1903-12-25 -1903-05-05 12:34:56.1476,1903-12-25 -1903-05-05 12:34:56.1477,1903-12-25 -1903-05-05 12:34:56.1478,1903-12-25 -1903-05-05 12:34:56.1479,1903-12-25 -1903-05-05 12:34:56.148,1903-12-25 -1903-05-05 12:34:56.1481,1903-12-25 -1903-05-05 12:34:56.1482,1903-12-25 -1903-05-05 12:34:56.1483,1903-12-25 -1903-05-05 12:34:56.1484,1903-12-25 -1903-05-05 12:34:56.1485,1903-12-25 -1903-05-05 12:34:56.1486,1903-12-25 -1903-05-05 12:34:56.1487,1903-12-25 -1903-05-05 12:34:56.1488,1903-12-25 -1903-05-05 12:34:56.1489,1903-12-25 -1903-05-05 12:34:56.149,1903-12-25 -1903-05-05 12:34:56.1491,1903-12-25 -1903-05-05 12:34:56.1492,1903-12-25 -1903-05-05 12:34:56.1493,1903-12-25 -1903-05-05 12:34:56.1494,1903-12-25 -1903-05-05 12:34:56.1495,1903-12-25 -1903-05-05 12:34:56.1496,1903-12-25 -1903-05-05 12:34:56.1497,1903-12-25 -1903-05-05 12:34:56.1498,1903-12-25 -1903-05-05 12:34:56.1499,1903-12-25 -1903-05-05 12:34:56.15,1903-12-25 -1903-05-05 12:34:56.1501,1903-12-25 -1903-05-05 12:34:56.1502,1903-12-25 -1903-05-05 12:34:56.1503,1903-12-25 -1903-05-05 12:34:56.1504,1903-12-25 -1903-05-05 12:34:56.1505,1903-12-25 -1903-05-05 12:34:56.1506,1903-12-25 -1903-05-05 12:34:56.1507,1903-12-25 -1903-05-05 12:34:56.1508,1903-12-25 -1903-05-05 12:34:56.1509,1903-12-25 -1903-05-05 12:34:56.151,1903-12-25 -1903-05-05 12:34:56.1511,1903-12-25 -1903-05-05 12:34:56.1512,1903-12-25 -1903-05-05 12:34:56.1513,1903-12-25 -1903-05-05 12:34:56.1514,1903-12-25 -1903-05-05 12:34:56.1515,1903-12-25 -1903-05-05 12:34:56.1516,1903-12-25 -1903-05-05 12:34:56.1517,1903-12-25 -1903-05-05 12:34:56.1518,1903-12-25 -1903-05-05 12:34:56.1519,1903-12-25 -1903-05-05 12:34:56.152,1903-12-25 -1903-05-05 12:34:56.1521,1903-12-25 -1903-05-05 12:34:56.1522,1903-12-25 -1903-05-05 12:34:56.1523,1903-12-25 -1903-05-05 12:34:56.1524,1903-12-25 -1903-05-05 12:34:56.1525,1903-12-25 -1903-05-05 12:34:56.1526,1903-12-25 -1903-05-05 12:34:56.1527,1903-12-25 -1903-05-05 12:34:56.1528,1903-12-25 -1903-05-05 12:34:56.1529,1903-12-25 -1903-05-05 12:34:56.153,1903-12-25 -1903-05-05 12:34:56.1531,1903-12-25 -1903-05-05 12:34:56.1532,1903-12-25 -1903-05-05 12:34:56.1533,1903-12-25 -1903-05-05 12:34:56.1534,1903-12-25 -1903-05-05 12:34:56.1535,1903-12-25 -1903-05-05 12:34:56.1536,1903-12-25 -1903-05-05 12:34:56.1537,1903-12-25 -1903-05-05 12:34:56.1538,1903-12-25 -1903-05-05 12:34:56.1539,1903-12-25 -1903-05-05 12:34:56.154,1903-12-25 -1903-05-05 12:34:56.1541,1903-12-25 -1903-05-05 12:34:56.1542,1903-12-25 -1903-05-05 12:34:56.1543,1903-12-25 -1903-05-05 12:34:56.1544,1903-12-25 -1903-05-05 12:34:56.1545,1903-12-25 -1903-05-05 12:34:56.1546,1903-12-25 -1903-05-05 12:34:56.1547,1903-12-25 -1903-05-05 12:34:56.1548,1903-12-25 -1903-05-05 12:34:56.1549,1903-12-25 -1903-05-05 12:34:56.155,1903-12-25 -1903-05-05 12:34:56.1551,1903-12-25 -1903-05-05 12:34:56.1552,1903-12-25 -1903-05-05 12:34:56.1553,1903-12-25 -1903-05-05 12:34:56.1554,1903-12-25 -1903-05-05 12:34:56.1555,1903-12-25 -1903-05-05 12:34:56.1556,1903-12-25 -1903-05-05 12:34:56.1557,1903-12-25 -1903-05-05 12:34:56.1558,1903-12-25 -1903-05-05 12:34:56.1559,1903-12-25 -1903-05-05 12:34:56.156,1903-12-25 -1903-05-05 12:34:56.1561,1903-12-25 -1903-05-05 12:34:56.1562,1903-12-25 -1903-05-05 12:34:56.1563,1903-12-25 -1903-05-05 12:34:56.1564,1903-12-25 -1903-05-05 12:34:56.1565,1903-12-25 -1903-05-05 12:34:56.1566,1903-12-25 -1903-05-05 12:34:56.1567,1903-12-25 -1903-05-05 12:34:56.1568,1903-12-25 -1903-05-05 12:34:56.1569,1903-12-25 -1903-05-05 12:34:56.157,1903-12-25 -1903-05-05 12:34:56.1571,1903-12-25 -1903-05-05 12:34:56.1572,1903-12-25 -1903-05-05 12:34:56.1573,1903-12-25 -1903-05-05 12:34:56.1574,1903-12-25 -1903-05-05 12:34:56.1575,1903-12-25 -1903-05-05 12:34:56.1576,1903-12-25 -1903-05-05 12:34:56.1577,1903-12-25 -1903-05-05 12:34:56.1578,1903-12-25 -1903-05-05 12:34:56.1579,1903-12-25 -1903-05-05 12:34:56.158,1903-12-25 -1903-05-05 12:34:56.1581,1903-12-25 -1903-05-05 12:34:56.1582,1903-12-25 -1903-05-05 12:34:56.1583,1903-12-25 -1903-05-05 12:34:56.1584,1903-12-25 -1903-05-05 12:34:56.1585,1903-12-25 -1903-05-05 12:34:56.1586,1903-12-25 -1903-05-05 12:34:56.1587,1903-12-25 -1903-05-05 12:34:56.1588,1903-12-25 -1903-05-05 12:34:56.1589,1903-12-25 -1903-05-05 12:34:56.159,1903-12-25 -1903-05-05 12:34:56.1591,1903-12-25 -1903-05-05 12:34:56.1592,1903-12-25 -1903-05-05 12:34:56.1593,1903-12-25 -1903-05-05 12:34:56.1594,1903-12-25 -1903-05-05 12:34:56.1595,1903-12-25 -1903-05-05 12:34:56.1596,1903-12-25 -1903-05-05 12:34:56.1597,1903-12-25 -1903-05-05 12:34:56.1598,1903-12-25 -1903-05-05 12:34:56.1599,1903-12-25 -1903-05-05 12:34:56.16,1903-12-25 -1903-05-05 12:34:56.1601,1903-12-25 -1903-05-05 12:34:56.1602,1903-12-25 -1903-05-05 12:34:56.1603,1903-12-25 -1903-05-05 12:34:56.1604,1903-12-25 -1903-05-05 12:34:56.1605,1903-12-25 -1903-05-05 12:34:56.1606,1903-12-25 -1903-05-05 12:34:56.1607,1903-12-25 -1903-05-05 12:34:56.1608,1903-12-25 -1903-05-05 12:34:56.1609,1903-12-25 -1903-05-05 12:34:56.161,1903-12-25 -1903-05-05 12:34:56.1611,1903-12-25 -1903-05-05 12:34:56.1612,1903-12-25 -1903-05-05 12:34:56.1613,1903-12-25 -1903-05-05 12:34:56.1614,1903-12-25 -1903-05-05 12:34:56.1615,1903-12-25 -1903-05-05 12:34:56.1616,1903-12-25 -1903-05-05 12:34:56.1617,1903-12-25 -1903-05-05 12:34:56.1618,1903-12-25 -1903-05-05 12:34:56.1619,1903-12-25 -1903-05-05 12:34:56.162,1903-12-25 -1903-05-05 12:34:56.1621,1903-12-25 -1903-05-05 12:34:56.1622,1903-12-25 -1903-05-05 12:34:56.1623,1903-12-25 -1903-05-05 12:34:56.1624,1903-12-25 -1903-05-05 12:34:56.1625,1903-12-25 -1903-05-05 12:34:56.1626,1903-12-25 -1903-05-05 12:34:56.1627,1903-12-25 -1903-05-05 12:34:56.1628,1903-12-25 -1903-05-05 12:34:56.1629,1903-12-25 -1903-05-05 12:34:56.163,1903-12-25 -1903-05-05 12:34:56.1631,1903-12-25 -1903-05-05 12:34:56.1632,1903-12-25 -1903-05-05 12:34:56.1633,1903-12-25 -1903-05-05 12:34:56.1634,1903-12-25 -1903-05-05 12:34:56.1635,1903-12-25 -1903-05-05 12:34:56.1636,1903-12-25 -1903-05-05 12:34:56.1637,1903-12-25 -1903-05-05 12:34:56.1638,1903-12-25 -1903-05-05 12:34:56.1639,1903-12-25 -1903-05-05 12:34:56.164,1903-12-25 -1903-05-05 12:34:56.1641,1903-12-25 -1903-05-05 12:34:56.1642,1903-12-25 -1903-05-05 12:34:56.1643,1903-12-25 -1903-05-05 12:34:56.1644,1903-12-25 -1903-05-05 12:34:56.1645,1903-12-25 -1903-05-05 12:34:56.1646,1903-12-25 -1903-05-05 12:34:56.1647,1903-12-25 -1903-05-05 12:34:56.1648,1903-12-25 -1903-05-05 12:34:56.1649,1903-12-25 -1903-05-05 12:34:56.165,1903-12-25 -1903-05-05 12:34:56.1651,1903-12-25 -1903-05-05 12:34:56.1652,1903-12-25 -1903-05-05 12:34:56.1653,1903-12-25 -1903-05-05 12:34:56.1654,1903-12-25 -1903-05-05 12:34:56.1655,1903-12-25 -1903-05-05 12:34:56.1656,1903-12-25 -1903-05-05 12:34:56.1657,1903-12-25 -1903-05-05 12:34:56.1658,1903-12-25 -1903-05-05 12:34:56.1659,1903-12-25 -1903-05-05 12:34:56.166,1903-12-25 -1903-05-05 12:34:56.1661,1903-12-25 -1903-05-05 12:34:56.1662,1903-12-25 -1903-05-05 12:34:56.1663,1903-12-25 -1903-05-05 12:34:56.1664,1903-12-25 -1903-05-05 12:34:56.1665,1903-12-25 -1903-05-05 12:34:56.1666,1903-12-25 -1903-05-05 12:34:56.1667,1903-12-25 -1903-05-05 12:34:56.1668,1903-12-25 -1903-05-05 12:34:56.1669,1903-12-25 -1903-05-05 12:34:56.167,1903-12-25 -1903-05-05 12:34:56.1671,1903-12-25 -1903-05-05 12:34:56.1672,1903-12-25 -1903-05-05 12:34:56.1673,1903-12-25 -1903-05-05 12:34:56.1674,1903-12-25 -1903-05-05 12:34:56.1675,1903-12-25 -1903-05-05 12:34:56.1676,1903-12-25 -1903-05-05 12:34:56.1677,1903-12-25 -1903-05-05 12:34:56.1678,1903-12-25 -1903-05-05 12:34:56.1679,1903-12-25 -1903-05-05 12:34:56.168,1903-12-25 -1903-05-05 12:34:56.1681,1903-12-25 -1903-05-05 12:34:56.1682,1903-12-25 -1903-05-05 12:34:56.1683,1903-12-25 -1903-05-05 12:34:56.1684,1903-12-25 -1903-05-05 12:34:56.1685,1903-12-25 -1903-05-05 12:34:56.1686,1903-12-25 -1903-05-05 12:34:56.1687,1903-12-25 -1903-05-05 12:34:56.1688,1903-12-25 -1903-05-05 12:34:56.1689,1903-12-25 -1903-05-05 12:34:56.169,1903-12-25 -1903-05-05 12:34:56.1691,1903-12-25 -1903-05-05 12:34:56.1692,1903-12-25 -1903-05-05 12:34:56.1693,1903-12-25 -1903-05-05 12:34:56.1694,1903-12-25 -1903-05-05 12:34:56.1695,1903-12-25 -1903-05-05 12:34:56.1696,1903-12-25 -1903-05-05 12:34:56.1697,1903-12-25 -1903-05-05 12:34:56.1698,1903-12-25 -1903-05-05 12:34:56.1699,1903-12-25 -1903-05-05 12:34:56.17,1903-12-25 -1903-05-05 12:34:56.1701,1903-12-25 -1903-05-05 12:34:56.1702,1903-12-25 -1903-05-05 12:34:56.1703,1903-12-25 -1903-05-05 12:34:56.1704,1903-12-25 -1903-05-05 12:34:56.1705,1903-12-25 -1903-05-05 12:34:56.1706,1903-12-25 -1903-05-05 12:34:56.1707,1903-12-25 -1903-05-05 12:34:56.1708,1903-12-25 -1903-05-05 12:34:56.1709,1903-12-25 -1903-05-05 12:34:56.171,1903-12-25 -1903-05-05 12:34:56.1711,1903-12-25 -1903-05-05 12:34:56.1712,1903-12-25 -1903-05-05 12:34:56.1713,1903-12-25 -1903-05-05 12:34:56.1714,1903-12-25 -1903-05-05 12:34:56.1715,1903-12-25 -1903-05-05 12:34:56.1716,1903-12-25 -1903-05-05 12:34:56.1717,1903-12-25 -1903-05-05 12:34:56.1718,1903-12-25 -1903-05-05 12:34:56.1719,1903-12-25 -1903-05-05 12:34:56.172,1903-12-25 -1903-05-05 12:34:56.1721,1903-12-25 -1903-05-05 12:34:56.1722,1903-12-25 -1903-05-05 12:34:56.1723,1903-12-25 -1903-05-05 12:34:56.1724,1903-12-25 -1903-05-05 12:34:56.1725,1903-12-25 -1903-05-05 12:34:56.1726,1903-12-25 -1903-05-05 12:34:56.1727,1903-12-25 -1903-05-05 12:34:56.1728,1903-12-25 -1903-05-05 12:34:56.1729,1903-12-25 -1903-05-05 12:34:56.173,1903-12-25 -1903-05-05 12:34:56.1731,1903-12-25 -1903-05-05 12:34:56.1732,1903-12-25 -1903-05-05 12:34:56.1733,1903-12-25 -1903-05-05 12:34:56.1734,1903-12-25 -1903-05-05 12:34:56.1735,1903-12-25 -1903-05-05 12:34:56.1736,1903-12-25 -1903-05-05 12:34:56.1737,1903-12-25 -1903-05-05 12:34:56.1738,1903-12-25 -1903-05-05 12:34:56.1739,1903-12-25 -1903-05-05 12:34:56.174,1903-12-25 -1903-05-05 12:34:56.1741,1903-12-25 -1903-05-05 12:34:56.1742,1903-12-25 -1903-05-05 12:34:56.1743,1903-12-25 -1903-05-05 12:34:56.1744,1903-12-25 -1903-05-05 12:34:56.1745,1903-12-25 -1903-05-05 12:34:56.1746,1903-12-25 -1903-05-05 12:34:56.1747,1903-12-25 -1903-05-05 12:34:56.1748,1903-12-25 -1903-05-05 12:34:56.1749,1903-12-25 -1903-05-05 12:34:56.175,1903-12-25 -1903-05-05 12:34:56.1751,1903-12-25 -1903-05-05 12:34:56.1752,1903-12-25 -1903-05-05 12:34:56.1753,1903-12-25 -1903-05-05 12:34:56.1754,1903-12-25 -1903-05-05 12:34:56.1755,1903-12-25 -1903-05-05 12:34:56.1756,1903-12-25 -1903-05-05 12:34:56.1757,1903-12-25 -1903-05-05 12:34:56.1758,1903-12-25 -1903-05-05 12:34:56.1759,1903-12-25 -1903-05-05 12:34:56.176,1903-12-25 -1903-05-05 12:34:56.1761,1903-12-25 -1903-05-05 12:34:56.1762,1903-12-25 -1903-05-05 12:34:56.1763,1903-12-25 -1903-05-05 12:34:56.1764,1903-12-25 -1903-05-05 12:34:56.1765,1903-12-25 -1903-05-05 12:34:56.1766,1903-12-25 -1903-05-05 12:34:56.1767,1903-12-25 -1903-05-05 12:34:56.1768,1903-12-25 -1903-05-05 12:34:56.1769,1903-12-25 -1903-05-05 12:34:56.177,1903-12-25 -1903-05-05 12:34:56.1771,1903-12-25 -1903-05-05 12:34:56.1772,1903-12-25 -1903-05-05 12:34:56.1773,1903-12-25 -1903-05-05 12:34:56.1774,1903-12-25 -1903-05-05 12:34:56.1775,1903-12-25 -1903-05-05 12:34:56.1776,1903-12-25 -1903-05-05 12:34:56.1777,1903-12-25 -1903-05-05 12:34:56.1778,1903-12-25 -1903-05-05 12:34:56.1779,1903-12-25 -1903-05-05 12:34:56.178,1903-12-25 -1903-05-05 12:34:56.1781,1903-12-25 -1903-05-05 12:34:56.1782,1903-12-25 -1903-05-05 12:34:56.1783,1903-12-25 -1903-05-05 12:34:56.1784,1903-12-25 -1903-05-05 12:34:56.1785,1903-12-25 -1903-05-05 12:34:56.1786,1903-12-25 -1903-05-05 12:34:56.1787,1903-12-25 -1903-05-05 12:34:56.1788,1903-12-25 -1903-05-05 12:34:56.1789,1903-12-25 -1903-05-05 12:34:56.179,1903-12-25 -1903-05-05 12:34:56.1791,1903-12-25 -1903-05-05 12:34:56.1792,1903-12-25 -1903-05-05 12:34:56.1793,1903-12-25 -1903-05-05 12:34:56.1794,1903-12-25 -1903-05-05 12:34:56.1795,1903-12-25 -1903-05-05 12:34:56.1796,1903-12-25 -1903-05-05 12:34:56.1797,1903-12-25 -1903-05-05 12:34:56.1798,1903-12-25 -1903-05-05 12:34:56.1799,1903-12-25 -1903-05-05 12:34:56.18,1903-12-25 -1903-05-05 12:34:56.1801,1903-12-25 -1903-05-05 12:34:56.1802,1903-12-25 -1903-05-05 12:34:56.1803,1903-12-25 -1903-05-05 12:34:56.1804,1903-12-25 -1903-05-05 12:34:56.1805,1903-12-25 -1903-05-05 12:34:56.1806,1903-12-25 -1903-05-05 12:34:56.1807,1903-12-25 -1903-05-05 12:34:56.1808,1903-12-25 -1903-05-05 12:34:56.1809,1903-12-25 -1903-05-05 12:34:56.181,1903-12-25 -1903-05-05 12:34:56.1811,1903-12-25 -1903-05-05 12:34:56.1812,1903-12-25 -1903-05-05 12:34:56.1813,1903-12-25 -1903-05-05 12:34:56.1814,1903-12-25 -1903-05-05 12:34:56.1815,1903-12-25 -1903-05-05 12:34:56.1816,1903-12-25 -1903-05-05 12:34:56.1817,1903-12-25 -1903-05-05 12:34:56.1818,1903-12-25 -1903-05-05 12:34:56.1819,1903-12-25 -1903-05-05 12:34:56.182,1903-12-25 -1903-05-05 12:34:56.1821,1903-12-25 -1903-05-05 12:34:56.1822,1903-12-25 -1903-05-05 12:34:56.1823,1903-12-25 -1903-05-05 12:34:56.1824,1903-12-25 -1903-05-05 12:34:56.1825,1903-12-25 -1903-05-05 12:34:56.1826,1903-12-25 -1903-05-05 12:34:56.1827,1903-12-25 -1903-05-05 12:34:56.1828,1903-12-25 -1903-05-05 12:34:56.1829,1903-12-25 -1903-05-05 12:34:56.183,1903-12-25 -1903-05-05 12:34:56.1831,1903-12-25 -1903-05-05 12:34:56.1832,1903-12-25 -1903-05-05 12:34:56.1833,1903-12-25 -1903-05-05 12:34:56.1834,1903-12-25 -1903-05-05 12:34:56.1835,1903-12-25 -1903-05-05 12:34:56.1836,1903-12-25 -1903-05-05 12:34:56.1837,1903-12-25 -1903-05-05 12:34:56.1838,1903-12-25 -1903-05-05 12:34:56.1839,1903-12-25 -1903-05-05 12:34:56.184,1903-12-25 -1903-05-05 12:34:56.1841,1903-12-25 -1903-05-05 12:34:56.1842,1903-12-25 -1903-05-05 12:34:56.1843,1903-12-25 -1903-05-05 12:34:56.1844,1903-12-25 -1903-05-05 12:34:56.1845,1903-12-25 -1903-05-05 12:34:56.1846,1903-12-25 -1903-05-05 12:34:56.1847,1903-12-25 -1903-05-05 12:34:56.1848,1903-12-25 -1903-05-05 12:34:56.1849,1903-12-25 -1903-05-05 12:34:56.185,1903-12-25 -1903-05-05 12:34:56.1851,1903-12-25 -1903-05-05 12:34:56.1852,1903-12-25 -1903-05-05 12:34:56.1853,1903-12-25 -1903-05-05 12:34:56.1854,1903-12-25 -1903-05-05 12:34:56.1855,1903-12-25 -1903-05-05 12:34:56.1856,1903-12-25 -1903-05-05 12:34:56.1857,1903-12-25 -1903-05-05 12:34:56.1858,1903-12-25 -1903-05-05 12:34:56.1859,1903-12-25 -1903-05-05 12:34:56.186,1903-12-25 -1903-05-05 12:34:56.1861,1903-12-25 -1903-05-05 12:34:56.1862,1903-12-25 -1903-05-05 12:34:56.1863,1903-12-25 -1903-05-05 12:34:56.1864,1903-12-25 -1903-05-05 12:34:56.1865,1903-12-25 -1903-05-05 12:34:56.1866,1903-12-25 -1903-05-05 12:34:56.1867,1903-12-25 -1903-05-05 12:34:56.1868,1903-12-25 -1903-05-05 12:34:56.1869,1903-12-25 -1903-05-05 12:34:56.187,1903-12-25 -1903-05-05 12:34:56.1871,1903-12-25 -1903-05-05 12:34:56.1872,1903-12-25 -1903-05-05 12:34:56.1873,1903-12-25 -1903-05-05 12:34:56.1874,1903-12-25 -1903-05-05 12:34:56.1875,1903-12-25 -1903-05-05 12:34:56.1876,1903-12-25 -1903-05-05 12:34:56.1877,1903-12-25 -1903-05-05 12:34:56.1878,1903-12-25 -1903-05-05 12:34:56.1879,1903-12-25 -1903-05-05 12:34:56.188,1903-12-25 -1903-05-05 12:34:56.1881,1903-12-25 -1903-05-05 12:34:56.1882,1903-12-25 -1903-05-05 12:34:56.1883,1903-12-25 -1903-05-05 12:34:56.1884,1903-12-25 -1903-05-05 12:34:56.1885,1903-12-25 -1903-05-05 12:34:56.1886,1903-12-25 -1903-05-05 12:34:56.1887,1903-12-25 -1903-05-05 12:34:56.1888,1903-12-25 -1903-05-05 12:34:56.1889,1903-12-25 -1903-05-05 12:34:56.189,1903-12-25 -1903-05-05 12:34:56.1891,1903-12-25 -1903-05-05 12:34:56.1892,1903-12-25 -1903-05-05 12:34:56.1893,1903-12-25 -1903-05-05 12:34:56.1894,1903-12-25 -1903-05-05 12:34:56.1895,1903-12-25 -1903-05-05 12:34:56.1896,1903-12-25 -1903-05-05 12:34:56.1897,1903-12-25 -1903-05-05 12:34:56.1898,1903-12-25 -1903-05-05 12:34:56.1899,1903-12-25 -1903-05-05 12:34:56.19,1903-12-25 -1903-05-05 12:34:56.1901,1903-12-25 -1903-05-05 12:34:56.1902,1903-12-25 -1903-05-05 12:34:56.1903,1903-12-25 -1903-05-05 12:34:56.1904,1903-12-25 -1903-05-05 12:34:56.1905,1903-12-25 -1903-05-05 12:34:56.1906,1903-12-25 -1903-05-05 12:34:56.1907,1903-12-25 -1903-05-05 12:34:56.1908,1903-12-25 -1903-05-05 12:34:56.1909,1903-12-25 -1903-05-05 12:34:56.191,1903-12-25 -1903-05-05 12:34:56.1911,1903-12-25 -1903-05-05 12:34:56.1912,1903-12-25 -1903-05-05 12:34:56.1913,1903-12-25 -1903-05-05 12:34:56.1914,1903-12-25 -1903-05-05 12:34:56.1915,1903-12-25 -1903-05-05 12:34:56.1916,1903-12-25 -1903-05-05 12:34:56.1917,1903-12-25 -1903-05-05 12:34:56.1918,1903-12-25 -1903-05-05 12:34:56.1919,1903-12-25 -1903-05-05 12:34:56.192,1903-12-25 -1903-05-05 12:34:56.1921,1903-12-25 -1903-05-05 12:34:56.1922,1903-12-25 -1903-05-05 12:34:56.1923,1903-12-25 -1903-05-05 12:34:56.1924,1903-12-25 -1903-05-05 12:34:56.1925,1903-12-25 -1903-05-05 12:34:56.1926,1903-12-25 -1903-05-05 12:34:56.1927,1903-12-25 -1903-05-05 12:34:56.1928,1903-12-25 -1903-05-05 12:34:56.1929,1903-12-25 -1903-05-05 12:34:56.193,1903-12-25 -1903-05-05 12:34:56.1931,1903-12-25 -1903-05-05 12:34:56.1932,1903-12-25 -1903-05-05 12:34:56.1933,1903-12-25 -1903-05-05 12:34:56.1934,1903-12-25 -1903-05-05 12:34:56.1935,1903-12-25 -1903-05-05 12:34:56.1936,1903-12-25 -1903-05-05 12:34:56.1937,1903-12-25 -1903-05-05 12:34:56.1938,1903-12-25 -1903-05-05 12:34:56.1939,1903-12-25 -1903-05-05 12:34:56.194,1903-12-25 -1903-05-05 12:34:56.1941,1903-12-25 -1903-05-05 12:34:56.1942,1903-12-25 -1903-05-05 12:34:56.1943,1903-12-25 -1903-05-05 12:34:56.1944,1903-12-25 -1903-05-05 12:34:56.1945,1903-12-25 -1903-05-05 12:34:56.1946,1903-12-25 -1903-05-05 12:34:56.1947,1903-12-25 -1903-05-05 12:34:56.1948,1903-12-25 -1903-05-05 12:34:56.1949,1903-12-25 -1903-05-05 12:34:56.195,1903-12-25 -1903-05-05 12:34:56.1951,1903-12-25 -1903-05-05 12:34:56.1952,1903-12-25 -1903-05-05 12:34:56.1953,1903-12-25 -1903-05-05 12:34:56.1954,1903-12-25 -1903-05-05 12:34:56.1955,1903-12-25 -1903-05-05 12:34:56.1956,1903-12-25 -1903-05-05 12:34:56.1957,1903-12-25 -1903-05-05 12:34:56.1958,1903-12-25 -1903-05-05 12:34:56.1959,1903-12-25 -1903-05-05 12:34:56.196,1903-12-25 -1903-05-05 12:34:56.1961,1903-12-25 -1903-05-05 12:34:56.1962,1903-12-25 -1903-05-05 12:34:56.1963,1903-12-25 -1903-05-05 12:34:56.1964,1903-12-25 -1903-05-05 12:34:56.1965,1903-12-25 -1903-05-05 12:34:56.1966,1903-12-25 -1903-05-05 12:34:56.1967,1903-12-25 -1903-05-05 12:34:56.1968,1903-12-25 -1903-05-05 12:34:56.1969,1903-12-25 -1903-05-05 12:34:56.197,1903-12-25 -1903-05-05 12:34:56.1971,1903-12-25 -1903-05-05 12:34:56.1972,1903-12-25 -1903-05-05 12:34:56.1973,1903-12-25 -1903-05-05 12:34:56.1974,1903-12-25 -1903-05-05 12:34:56.1975,1903-12-25 -1903-05-05 12:34:56.1976,1903-12-25 -1903-05-05 12:34:56.1977,1903-12-25 -1903-05-05 12:34:56.1978,1903-12-25 -1903-05-05 12:34:56.1979,1903-12-25 -1903-05-05 12:34:56.198,1903-12-25 -1903-05-05 12:34:56.1981,1903-12-25 -1903-05-05 12:34:56.1982,1903-12-25 -1903-05-05 12:34:56.1983,1903-12-25 -1903-05-05 12:34:56.1984,1903-12-25 -1903-05-05 12:34:56.1985,1903-12-25 -1903-05-05 12:34:56.1986,1903-12-25 -1903-05-05 12:34:56.1987,1903-12-25 -1903-05-05 12:34:56.1988,1903-12-25 -1903-05-05 12:34:56.1989,1903-12-25 -1903-05-05 12:34:56.199,1903-12-25 -1903-05-05 12:34:56.1991,1903-12-25 -1903-05-05 12:34:56.1992,1903-12-25 -1903-05-05 12:34:56.1993,1903-12-25 -1903-05-05 12:34:56.1994,1903-12-25 -1903-05-05 12:34:56.1995,1903-12-25 -1903-05-05 12:34:56.1996,1903-12-25 -1903-05-05 12:34:56.1997,1903-12-25 -1903-05-05 12:34:56.1998,1903-12-25 -1903-05-05 12:34:56.1999,1903-12-25 -1904-05-05 12:34:56.1,1904-12-25 -1904-05-05 12:34:56.1001,1904-12-25 -1904-05-05 12:34:56.1002,1904-12-25 -1904-05-05 12:34:56.1003,1904-12-25 -1904-05-05 12:34:56.1004,1904-12-25 -1904-05-05 12:34:56.1005,1904-12-25 -1904-05-05 12:34:56.1006,1904-12-25 -1904-05-05 12:34:56.1007,1904-12-25 -1904-05-05 12:34:56.1008,1904-12-25 -1904-05-05 12:34:56.1009,1904-12-25 -1904-05-05 12:34:56.101,1904-12-25 -1904-05-05 12:34:56.1011,1904-12-25 -1904-05-05 12:34:56.1012,1904-12-25 -1904-05-05 12:34:56.1013,1904-12-25 -1904-05-05 12:34:56.1014,1904-12-25 -1904-05-05 12:34:56.1015,1904-12-25 -1904-05-05 12:34:56.1016,1904-12-25 -1904-05-05 12:34:56.1017,1904-12-25 -1904-05-05 12:34:56.1018,1904-12-25 -1904-05-05 12:34:56.1019,1904-12-25 -1904-05-05 12:34:56.102,1904-12-25 -1904-05-05 12:34:56.1021,1904-12-25 -1904-05-05 12:34:56.1022,1904-12-25 -1904-05-05 12:34:56.1023,1904-12-25 -1904-05-05 12:34:56.1024,1904-12-25 -1904-05-05 12:34:56.1025,1904-12-25 -1904-05-05 12:34:56.1026,1904-12-25 -1904-05-05 12:34:56.1027,1904-12-25 -1904-05-05 12:34:56.1028,1904-12-25 -1904-05-05 12:34:56.1029,1904-12-25 -1904-05-05 12:34:56.103,1904-12-25 -1904-05-05 12:34:56.1031,1904-12-25 -1904-05-05 12:34:56.1032,1904-12-25 -1904-05-05 12:34:56.1033,1904-12-25 -1904-05-05 12:34:56.1034,1904-12-25 -1904-05-05 12:34:56.1035,1904-12-25 -1904-05-05 12:34:56.1036,1904-12-25 -1904-05-05 12:34:56.1037,1904-12-25 -1904-05-05 12:34:56.1038,1904-12-25 -1904-05-05 12:34:56.1039,1904-12-25 -1904-05-05 12:34:56.104,1904-12-25 -1904-05-05 12:34:56.1041,1904-12-25 -1904-05-05 12:34:56.1042,1904-12-25 -1904-05-05 12:34:56.1043,1904-12-25 -1904-05-05 12:34:56.1044,1904-12-25 -1904-05-05 12:34:56.1045,1904-12-25 -1904-05-05 12:34:56.1046,1904-12-25 -1904-05-05 12:34:56.1047,1904-12-25 -1904-05-05 12:34:56.1048,1904-12-25 -1904-05-05 12:34:56.1049,1904-12-25 -1904-05-05 12:34:56.105,1904-12-25 -1904-05-05 12:34:56.1051,1904-12-25 -1904-05-05 12:34:56.1052,1904-12-25 -1904-05-05 12:34:56.1053,1904-12-25 -1904-05-05 12:34:56.1054,1904-12-25 -1904-05-05 12:34:56.1055,1904-12-25 -1904-05-05 12:34:56.1056,1904-12-25 -1904-05-05 12:34:56.1057,1904-12-25 -1904-05-05 12:34:56.1058,1904-12-25 -1904-05-05 12:34:56.1059,1904-12-25 -1904-05-05 12:34:56.106,1904-12-25 -1904-05-05 12:34:56.1061,1904-12-25 -1904-05-05 12:34:56.1062,1904-12-25 -1904-05-05 12:34:56.1063,1904-12-25 -1904-05-05 12:34:56.1064,1904-12-25 -1904-05-05 12:34:56.1065,1904-12-25 -1904-05-05 12:34:56.1066,1904-12-25 -1904-05-05 12:34:56.1067,1904-12-25 -1904-05-05 12:34:56.1068,1904-12-25 -1904-05-05 12:34:56.1069,1904-12-25 -1904-05-05 12:34:56.107,1904-12-25 -1904-05-05 12:34:56.1071,1904-12-25 -1904-05-05 12:34:56.1072,1904-12-25 -1904-05-05 12:34:56.1073,1904-12-25 -1904-05-05 12:34:56.1074,1904-12-25 -1904-05-05 12:34:56.1075,1904-12-25 -1904-05-05 12:34:56.1076,1904-12-25 -1904-05-05 12:34:56.1077,1904-12-25 -1904-05-05 12:34:56.1078,1904-12-25 -1904-05-05 12:34:56.1079,1904-12-25 -1904-05-05 12:34:56.108,1904-12-25 -1904-05-05 12:34:56.1081,1904-12-25 -1904-05-05 12:34:56.1082,1904-12-25 -1904-05-05 12:34:56.1083,1904-12-25 -1904-05-05 12:34:56.1084,1904-12-25 -1904-05-05 12:34:56.1085,1904-12-25 -1904-05-05 12:34:56.1086,1904-12-25 -1904-05-05 12:34:56.1087,1904-12-25 -1904-05-05 12:34:56.1088,1904-12-25 -1904-05-05 12:34:56.1089,1904-12-25 -1904-05-05 12:34:56.109,1904-12-25 -1904-05-05 12:34:56.1091,1904-12-25 -1904-05-05 12:34:56.1092,1904-12-25 -1904-05-05 12:34:56.1093,1904-12-25 -1904-05-05 12:34:56.1094,1904-12-25 -1904-05-05 12:34:56.1095,1904-12-25 -1904-05-05 12:34:56.1096,1904-12-25 -1904-05-05 12:34:56.1097,1904-12-25 -1904-05-05 12:34:56.1098,1904-12-25 -1904-05-05 12:34:56.1099,1904-12-25 -1904-05-05 12:34:56.11,1904-12-25 -1904-05-05 12:34:56.1101,1904-12-25 -1904-05-05 12:34:56.1102,1904-12-25 -1904-05-05 12:34:56.1103,1904-12-25 -1904-05-05 12:34:56.1104,1904-12-25 -1904-05-05 12:34:56.1105,1904-12-25 -1904-05-05 12:34:56.1106,1904-12-25 -1904-05-05 12:34:56.1107,1904-12-25 -1904-05-05 12:34:56.1108,1904-12-25 -1904-05-05 12:34:56.1109,1904-12-25 -1904-05-05 12:34:56.111,1904-12-25 -1904-05-05 12:34:56.1111,1904-12-25 -1904-05-05 12:34:56.1112,1904-12-25 -1904-05-05 12:34:56.1113,1904-12-25 -1904-05-05 12:34:56.1114,1904-12-25 -1904-05-05 12:34:56.1115,1904-12-25 -1904-05-05 12:34:56.1116,1904-12-25 -1904-05-05 12:34:56.1117,1904-12-25 -1904-05-05 12:34:56.1118,1904-12-25 -1904-05-05 12:34:56.1119,1904-12-25 -1904-05-05 12:34:56.112,1904-12-25 -1904-05-05 12:34:56.1121,1904-12-25 -1904-05-05 12:34:56.1122,1904-12-25 -1904-05-05 12:34:56.1123,1904-12-25 -1904-05-05 12:34:56.1124,1904-12-25 -1904-05-05 12:34:56.1125,1904-12-25 -1904-05-05 12:34:56.1126,1904-12-25 -1904-05-05 12:34:56.1127,1904-12-25 -1904-05-05 12:34:56.1128,1904-12-25 -1904-05-05 12:34:56.1129,1904-12-25 -1904-05-05 12:34:56.113,1904-12-25 -1904-05-05 12:34:56.1131,1904-12-25 -1904-05-05 12:34:56.1132,1904-12-25 -1904-05-05 12:34:56.1133,1904-12-25 -1904-05-05 12:34:56.1134,1904-12-25 -1904-05-05 12:34:56.1135,1904-12-25 -1904-05-05 12:34:56.1136,1904-12-25 -1904-05-05 12:34:56.1137,1904-12-25 -1904-05-05 12:34:56.1138,1904-12-25 -1904-05-05 12:34:56.1139,1904-12-25 -1904-05-05 12:34:56.114,1904-12-25 -1904-05-05 12:34:56.1141,1904-12-25 -1904-05-05 12:34:56.1142,1904-12-25 -1904-05-05 12:34:56.1143,1904-12-25 -1904-05-05 12:34:56.1144,1904-12-25 -1904-05-05 12:34:56.1145,1904-12-25 -1904-05-05 12:34:56.1146,1904-12-25 -1904-05-05 12:34:56.1147,1904-12-25 -1904-05-05 12:34:56.1148,1904-12-25 -1904-05-05 12:34:56.1149,1904-12-25 -1904-05-05 12:34:56.115,1904-12-25 -1904-05-05 12:34:56.1151,1904-12-25 -1904-05-05 12:34:56.1152,1904-12-25 -1904-05-05 12:34:56.1153,1904-12-25 -1904-05-05 12:34:56.1154,1904-12-25 -1904-05-05 12:34:56.1155,1904-12-25 -1904-05-05 12:34:56.1156,1904-12-25 -1904-05-05 12:34:56.1157,1904-12-25 -1904-05-05 12:34:56.1158,1904-12-25 -1904-05-05 12:34:56.1159,1904-12-25 -1904-05-05 12:34:56.116,1904-12-25 -1904-05-05 12:34:56.1161,1904-12-25 -1904-05-05 12:34:56.1162,1904-12-25 -1904-05-05 12:34:56.1163,1904-12-25 -1904-05-05 12:34:56.1164,1904-12-25 -1904-05-05 12:34:56.1165,1904-12-25 -1904-05-05 12:34:56.1166,1904-12-25 -1904-05-05 12:34:56.1167,1904-12-25 -1904-05-05 12:34:56.1168,1904-12-25 -1904-05-05 12:34:56.1169,1904-12-25 -1904-05-05 12:34:56.117,1904-12-25 -1904-05-05 12:34:56.1171,1904-12-25 -1904-05-05 12:34:56.1172,1904-12-25 -1904-05-05 12:34:56.1173,1904-12-25 -1904-05-05 12:34:56.1174,1904-12-25 -1904-05-05 12:34:56.1175,1904-12-25 -1904-05-05 12:34:56.1176,1904-12-25 -1904-05-05 12:34:56.1177,1904-12-25 -1904-05-05 12:34:56.1178,1904-12-25 -1904-05-05 12:34:56.1179,1904-12-25 -1904-05-05 12:34:56.118,1904-12-25 -1904-05-05 12:34:56.1181,1904-12-25 -1904-05-05 12:34:56.1182,1904-12-25 -1904-05-05 12:34:56.1183,1904-12-25 -1904-05-05 12:34:56.1184,1904-12-25 -1904-05-05 12:34:56.1185,1904-12-25 -1904-05-05 12:34:56.1186,1904-12-25 -1904-05-05 12:34:56.1187,1904-12-25 -1904-05-05 12:34:56.1188,1904-12-25 -1904-05-05 12:34:56.1189,1904-12-25 -1904-05-05 12:34:56.119,1904-12-25 -1904-05-05 12:34:56.1191,1904-12-25 -1904-05-05 12:34:56.1192,1904-12-25 -1904-05-05 12:34:56.1193,1904-12-25 -1904-05-05 12:34:56.1194,1904-12-25 -1904-05-05 12:34:56.1195,1904-12-25 -1904-05-05 12:34:56.1196,1904-12-25 -1904-05-05 12:34:56.1197,1904-12-25 -1904-05-05 12:34:56.1198,1904-12-25 -1904-05-05 12:34:56.1199,1904-12-25 -1904-05-05 12:34:56.12,1904-12-25 -1904-05-05 12:34:56.1201,1904-12-25 -1904-05-05 12:34:56.1202,1904-12-25 -1904-05-05 12:34:56.1203,1904-12-25 -1904-05-05 12:34:56.1204,1904-12-25 -1904-05-05 12:34:56.1205,1904-12-25 -1904-05-05 12:34:56.1206,1904-12-25 -1904-05-05 12:34:56.1207,1904-12-25 -1904-05-05 12:34:56.1208,1904-12-25 -1904-05-05 12:34:56.1209,1904-12-25 -1904-05-05 12:34:56.121,1904-12-25 -1904-05-05 12:34:56.1211,1904-12-25 -1904-05-05 12:34:56.1212,1904-12-25 -1904-05-05 12:34:56.1213,1904-12-25 -1904-05-05 12:34:56.1214,1904-12-25 -1904-05-05 12:34:56.1215,1904-12-25 -1904-05-05 12:34:56.1216,1904-12-25 -1904-05-05 12:34:56.1217,1904-12-25 -1904-05-05 12:34:56.1218,1904-12-25 -1904-05-05 12:34:56.1219,1904-12-25 -1904-05-05 12:34:56.122,1904-12-25 -1904-05-05 12:34:56.1221,1904-12-25 -1904-05-05 12:34:56.1222,1904-12-25 -1904-05-05 12:34:56.1223,1904-12-25 -1904-05-05 12:34:56.1224,1904-12-25 -1904-05-05 12:34:56.1225,1904-12-25 -1904-05-05 12:34:56.1226,1904-12-25 -1904-05-05 12:34:56.1227,1904-12-25 -1904-05-05 12:34:56.1228,1904-12-25 -1904-05-05 12:34:56.1229,1904-12-25 -1904-05-05 12:34:56.123,1904-12-25 -1904-05-05 12:34:56.1231,1904-12-25 -1904-05-05 12:34:56.1232,1904-12-25 -1904-05-05 12:34:56.1233,1904-12-25 -1904-05-05 12:34:56.1234,1904-12-25 -1904-05-05 12:34:56.1235,1904-12-25 -1904-05-05 12:34:56.1236,1904-12-25 -1904-05-05 12:34:56.1237,1904-12-25 -1904-05-05 12:34:56.1238,1904-12-25 -1904-05-05 12:34:56.1239,1904-12-25 -1904-05-05 12:34:56.124,1904-12-25 -1904-05-05 12:34:56.1241,1904-12-25 -1904-05-05 12:34:56.1242,1904-12-25 -1904-05-05 12:34:56.1243,1904-12-25 -1904-05-05 12:34:56.1244,1904-12-25 -1904-05-05 12:34:56.1245,1904-12-25 -1904-05-05 12:34:56.1246,1904-12-25 -1904-05-05 12:34:56.1247,1904-12-25 -1904-05-05 12:34:56.1248,1904-12-25 -1904-05-05 12:34:56.1249,1904-12-25 -1904-05-05 12:34:56.125,1904-12-25 -1904-05-05 12:34:56.1251,1904-12-25 -1904-05-05 12:34:56.1252,1904-12-25 -1904-05-05 12:34:56.1253,1904-12-25 -1904-05-05 12:34:56.1254,1904-12-25 -1904-05-05 12:34:56.1255,1904-12-25 -1904-05-05 12:34:56.1256,1904-12-25 -1904-05-05 12:34:56.1257,1904-12-25 -1904-05-05 12:34:56.1258,1904-12-25 -1904-05-05 12:34:56.1259,1904-12-25 -1904-05-05 12:34:56.126,1904-12-25 -1904-05-05 12:34:56.1261,1904-12-25 -1904-05-05 12:34:56.1262,1904-12-25 -1904-05-05 12:34:56.1263,1904-12-25 -1904-05-05 12:34:56.1264,1904-12-25 -1904-05-05 12:34:56.1265,1904-12-25 -1904-05-05 12:34:56.1266,1904-12-25 -1904-05-05 12:34:56.1267,1904-12-25 -1904-05-05 12:34:56.1268,1904-12-25 -1904-05-05 12:34:56.1269,1904-12-25 -1904-05-05 12:34:56.127,1904-12-25 -1904-05-05 12:34:56.1271,1904-12-25 -1904-05-05 12:34:56.1272,1904-12-25 -1904-05-05 12:34:56.1273,1904-12-25 -1904-05-05 12:34:56.1274,1904-12-25 -1904-05-05 12:34:56.1275,1904-12-25 -1904-05-05 12:34:56.1276,1904-12-25 -1904-05-05 12:34:56.1277,1904-12-25 -1904-05-05 12:34:56.1278,1904-12-25 -1904-05-05 12:34:56.1279,1904-12-25 -1904-05-05 12:34:56.128,1904-12-25 -1904-05-05 12:34:56.1281,1904-12-25 -1904-05-05 12:34:56.1282,1904-12-25 -1904-05-05 12:34:56.1283,1904-12-25 -1904-05-05 12:34:56.1284,1904-12-25 -1904-05-05 12:34:56.1285,1904-12-25 -1904-05-05 12:34:56.1286,1904-12-25 -1904-05-05 12:34:56.1287,1904-12-25 -1904-05-05 12:34:56.1288,1904-12-25 -1904-05-05 12:34:56.1289,1904-12-25 -1904-05-05 12:34:56.129,1904-12-25 -1904-05-05 12:34:56.1291,1904-12-25 -1904-05-05 12:34:56.1292,1904-12-25 -1904-05-05 12:34:56.1293,1904-12-25 -1904-05-05 12:34:56.1294,1904-12-25 -1904-05-05 12:34:56.1295,1904-12-25 -1904-05-05 12:34:56.1296,1904-12-25 -1904-05-05 12:34:56.1297,1904-12-25 -1904-05-05 12:34:56.1298,1904-12-25 -1904-05-05 12:34:56.1299,1904-12-25 -1904-05-05 12:34:56.13,1904-12-25 -1904-05-05 12:34:56.1301,1904-12-25 -1904-05-05 12:34:56.1302,1904-12-25 -1904-05-05 12:34:56.1303,1904-12-25 -1904-05-05 12:34:56.1304,1904-12-25 -1904-05-05 12:34:56.1305,1904-12-25 -1904-05-05 12:34:56.1306,1904-12-25 -1904-05-05 12:34:56.1307,1904-12-25 -1904-05-05 12:34:56.1308,1904-12-25 -1904-05-05 12:34:56.1309,1904-12-25 -1904-05-05 12:34:56.131,1904-12-25 -1904-05-05 12:34:56.1311,1904-12-25 -1904-05-05 12:34:56.1312,1904-12-25 -1904-05-05 12:34:56.1313,1904-12-25 -1904-05-05 12:34:56.1314,1904-12-25 -1904-05-05 12:34:56.1315,1904-12-25 -1904-05-05 12:34:56.1316,1904-12-25 -1904-05-05 12:34:56.1317,1904-12-25 -1904-05-05 12:34:56.1318,1904-12-25 -1904-05-05 12:34:56.1319,1904-12-25 -1904-05-05 12:34:56.132,1904-12-25 -1904-05-05 12:34:56.1321,1904-12-25 -1904-05-05 12:34:56.1322,1904-12-25 -1904-05-05 12:34:56.1323,1904-12-25 -1904-05-05 12:34:56.1324,1904-12-25 -1904-05-05 12:34:56.1325,1904-12-25 -1904-05-05 12:34:56.1326,1904-12-25 -1904-05-05 12:34:56.1327,1904-12-25 -1904-05-05 12:34:56.1328,1904-12-25 -1904-05-05 12:34:56.1329,1904-12-25 -1904-05-05 12:34:56.133,1904-12-25 -1904-05-05 12:34:56.1331,1904-12-25 -1904-05-05 12:34:56.1332,1904-12-25 -1904-05-05 12:34:56.1333,1904-12-25 -1904-05-05 12:34:56.1334,1904-12-25 -1904-05-05 12:34:56.1335,1904-12-25 -1904-05-05 12:34:56.1336,1904-12-25 -1904-05-05 12:34:56.1337,1904-12-25 -1904-05-05 12:34:56.1338,1904-12-25 -1904-05-05 12:34:56.1339,1904-12-25 -1904-05-05 12:34:56.134,1904-12-25 -1904-05-05 12:34:56.1341,1904-12-25 -1904-05-05 12:34:56.1342,1904-12-25 -1904-05-05 12:34:56.1343,1904-12-25 -1904-05-05 12:34:56.1344,1904-12-25 -1904-05-05 12:34:56.1345,1904-12-25 -1904-05-05 12:34:56.1346,1904-12-25 -1904-05-05 12:34:56.1347,1904-12-25 -1904-05-05 12:34:56.1348,1904-12-25 -1904-05-05 12:34:56.1349,1904-12-25 -1904-05-05 12:34:56.135,1904-12-25 -1904-05-05 12:34:56.1351,1904-12-25 -1904-05-05 12:34:56.1352,1904-12-25 -1904-05-05 12:34:56.1353,1904-12-25 -1904-05-05 12:34:56.1354,1904-12-25 -1904-05-05 12:34:56.1355,1904-12-25 -1904-05-05 12:34:56.1356,1904-12-25 -1904-05-05 12:34:56.1357,1904-12-25 -1904-05-05 12:34:56.1358,1904-12-25 -1904-05-05 12:34:56.1359,1904-12-25 -1904-05-05 12:34:56.136,1904-12-25 -1904-05-05 12:34:56.1361,1904-12-25 -1904-05-05 12:34:56.1362,1904-12-25 -1904-05-05 12:34:56.1363,1904-12-25 -1904-05-05 12:34:56.1364,1904-12-25 -1904-05-05 12:34:56.1365,1904-12-25 -1904-05-05 12:34:56.1366,1904-12-25 -1904-05-05 12:34:56.1367,1904-12-25 -1904-05-05 12:34:56.1368,1904-12-25 -1904-05-05 12:34:56.1369,1904-12-25 -1904-05-05 12:34:56.137,1904-12-25 -1904-05-05 12:34:56.1371,1904-12-25 -1904-05-05 12:34:56.1372,1904-12-25 -1904-05-05 12:34:56.1373,1904-12-25 -1904-05-05 12:34:56.1374,1904-12-25 -1904-05-05 12:34:56.1375,1904-12-25 -1904-05-05 12:34:56.1376,1904-12-25 -1904-05-05 12:34:56.1377,1904-12-25 -1904-05-05 12:34:56.1378,1904-12-25 -1904-05-05 12:34:56.1379,1904-12-25 -1904-05-05 12:34:56.138,1904-12-25 -1904-05-05 12:34:56.1381,1904-12-25 -1904-05-05 12:34:56.1382,1904-12-25 -1904-05-05 12:34:56.1383,1904-12-25 -1904-05-05 12:34:56.1384,1904-12-25 -1904-05-05 12:34:56.1385,1904-12-25 -1904-05-05 12:34:56.1386,1904-12-25 -1904-05-05 12:34:56.1387,1904-12-25 -1904-05-05 12:34:56.1388,1904-12-25 -1904-05-05 12:34:56.1389,1904-12-25 -1904-05-05 12:34:56.139,1904-12-25 -1904-05-05 12:34:56.1391,1904-12-25 -1904-05-05 12:34:56.1392,1904-12-25 -1904-05-05 12:34:56.1393,1904-12-25 -1904-05-05 12:34:56.1394,1904-12-25 -1904-05-05 12:34:56.1395,1904-12-25 -1904-05-05 12:34:56.1396,1904-12-25 -1904-05-05 12:34:56.1397,1904-12-25 -1904-05-05 12:34:56.1398,1904-12-25 -1904-05-05 12:34:56.1399,1904-12-25 -1904-05-05 12:34:56.14,1904-12-25 -1904-05-05 12:34:56.1401,1904-12-25 -1904-05-05 12:34:56.1402,1904-12-25 -1904-05-05 12:34:56.1403,1904-12-25 -1904-05-05 12:34:56.1404,1904-12-25 -1904-05-05 12:34:56.1405,1904-12-25 -1904-05-05 12:34:56.1406,1904-12-25 -1904-05-05 12:34:56.1407,1904-12-25 -1904-05-05 12:34:56.1408,1904-12-25 -1904-05-05 12:34:56.1409,1904-12-25 -1904-05-05 12:34:56.141,1904-12-25 -1904-05-05 12:34:56.1411,1904-12-25 -1904-05-05 12:34:56.1412,1904-12-25 -1904-05-05 12:34:56.1413,1904-12-25 -1904-05-05 12:34:56.1414,1904-12-25 -1904-05-05 12:34:56.1415,1904-12-25 -1904-05-05 12:34:56.1416,1904-12-25 -1904-05-05 12:34:56.1417,1904-12-25 -1904-05-05 12:34:56.1418,1904-12-25 -1904-05-05 12:34:56.1419,1904-12-25 -1904-05-05 12:34:56.142,1904-12-25 -1904-05-05 12:34:56.1421,1904-12-25 -1904-05-05 12:34:56.1422,1904-12-25 -1904-05-05 12:34:56.1423,1904-12-25 -1904-05-05 12:34:56.1424,1904-12-25 -1904-05-05 12:34:56.1425,1904-12-25 -1904-05-05 12:34:56.1426,1904-12-25 -1904-05-05 12:34:56.1427,1904-12-25 -1904-05-05 12:34:56.1428,1904-12-25 -1904-05-05 12:34:56.1429,1904-12-25 -1904-05-05 12:34:56.143,1904-12-25 -1904-05-05 12:34:56.1431,1904-12-25 -1904-05-05 12:34:56.1432,1904-12-25 -1904-05-05 12:34:56.1433,1904-12-25 -1904-05-05 12:34:56.1434,1904-12-25 -1904-05-05 12:34:56.1435,1904-12-25 -1904-05-05 12:34:56.1436,1904-12-25 -1904-05-05 12:34:56.1437,1904-12-25 -1904-05-05 12:34:56.1438,1904-12-25 -1904-05-05 12:34:56.1439,1904-12-25 -1904-05-05 12:34:56.144,1904-12-25 -1904-05-05 12:34:56.1441,1904-12-25 -1904-05-05 12:34:56.1442,1904-12-25 -1904-05-05 12:34:56.1443,1904-12-25 -1904-05-05 12:34:56.1444,1904-12-25 -1904-05-05 12:34:56.1445,1904-12-25 -1904-05-05 12:34:56.1446,1904-12-25 -1904-05-05 12:34:56.1447,1904-12-25 -1904-05-05 12:34:56.1448,1904-12-25 -1904-05-05 12:34:56.1449,1904-12-25 -1904-05-05 12:34:56.145,1904-12-25 -1904-05-05 12:34:56.1451,1904-12-25 -1904-05-05 12:34:56.1452,1904-12-25 -1904-05-05 12:34:56.1453,1904-12-25 -1904-05-05 12:34:56.1454,1904-12-25 -1904-05-05 12:34:56.1455,1904-12-25 -1904-05-05 12:34:56.1456,1904-12-25 -1904-05-05 12:34:56.1457,1904-12-25 -1904-05-05 12:34:56.1458,1904-12-25 -1904-05-05 12:34:56.1459,1904-12-25 -1904-05-05 12:34:56.146,1904-12-25 -1904-05-05 12:34:56.1461,1904-12-25 -1904-05-05 12:34:56.1462,1904-12-25 -1904-05-05 12:34:56.1463,1904-12-25 -1904-05-05 12:34:56.1464,1904-12-25 -1904-05-05 12:34:56.1465,1904-12-25 -1904-05-05 12:34:56.1466,1904-12-25 -1904-05-05 12:34:56.1467,1904-12-25 -1904-05-05 12:34:56.1468,1904-12-25 -1904-05-05 12:34:56.1469,1904-12-25 -1904-05-05 12:34:56.147,1904-12-25 -1904-05-05 12:34:56.1471,1904-12-25 -1904-05-05 12:34:56.1472,1904-12-25 -1904-05-05 12:34:56.1473,1904-12-25 -1904-05-05 12:34:56.1474,1904-12-25 -1904-05-05 12:34:56.1475,1904-12-25 -1904-05-05 12:34:56.1476,1904-12-25 -1904-05-05 12:34:56.1477,1904-12-25 -1904-05-05 12:34:56.1478,1904-12-25 -1904-05-05 12:34:56.1479,1904-12-25 -1904-05-05 12:34:56.148,1904-12-25 -1904-05-05 12:34:56.1481,1904-12-25 -1904-05-05 12:34:56.1482,1904-12-25 -1904-05-05 12:34:56.1483,1904-12-25 -1904-05-05 12:34:56.1484,1904-12-25 -1904-05-05 12:34:56.1485,1904-12-25 -1904-05-05 12:34:56.1486,1904-12-25 -1904-05-05 12:34:56.1487,1904-12-25 -1904-05-05 12:34:56.1488,1904-12-25 -1904-05-05 12:34:56.1489,1904-12-25 -1904-05-05 12:34:56.149,1904-12-25 -1904-05-05 12:34:56.1491,1904-12-25 -1904-05-05 12:34:56.1492,1904-12-25 -1904-05-05 12:34:56.1493,1904-12-25 -1904-05-05 12:34:56.1494,1904-12-25 -1904-05-05 12:34:56.1495,1904-12-25 -1904-05-05 12:34:56.1496,1904-12-25 -1904-05-05 12:34:56.1497,1904-12-25 -1904-05-05 12:34:56.1498,1904-12-25 -1904-05-05 12:34:56.1499,1904-12-25 -1904-05-05 12:34:56.15,1904-12-25 -1904-05-05 12:34:56.1501,1904-12-25 -1904-05-05 12:34:56.1502,1904-12-25 -1904-05-05 12:34:56.1503,1904-12-25 -1904-05-05 12:34:56.1504,1904-12-25 -1904-05-05 12:34:56.1505,1904-12-25 -1904-05-05 12:34:56.1506,1904-12-25 -1904-05-05 12:34:56.1507,1904-12-25 -1904-05-05 12:34:56.1508,1904-12-25 -1904-05-05 12:34:56.1509,1904-12-25 -1904-05-05 12:34:56.151,1904-12-25 -1904-05-05 12:34:56.1511,1904-12-25 -1904-05-05 12:34:56.1512,1904-12-25 -1904-05-05 12:34:56.1513,1904-12-25 -1904-05-05 12:34:56.1514,1904-12-25 -1904-05-05 12:34:56.1515,1904-12-25 -1904-05-05 12:34:56.1516,1904-12-25 -1904-05-05 12:34:56.1517,1904-12-25 -1904-05-05 12:34:56.1518,1904-12-25 -1904-05-05 12:34:56.1519,1904-12-25 -1904-05-05 12:34:56.152,1904-12-25 -1904-05-05 12:34:56.1521,1904-12-25 -1904-05-05 12:34:56.1522,1904-12-25 -1904-05-05 12:34:56.1523,1904-12-25 -1904-05-05 12:34:56.1524,1904-12-25 -1904-05-05 12:34:56.1525,1904-12-25 -1904-05-05 12:34:56.1526,1904-12-25 -1904-05-05 12:34:56.1527,1904-12-25 -1904-05-05 12:34:56.1528,1904-12-25 -1904-05-05 12:34:56.1529,1904-12-25 -1904-05-05 12:34:56.153,1904-12-25 -1904-05-05 12:34:56.1531,1904-12-25 -1904-05-05 12:34:56.1532,1904-12-25 -1904-05-05 12:34:56.1533,1904-12-25 -1904-05-05 12:34:56.1534,1904-12-25 -1904-05-05 12:34:56.1535,1904-12-25 -1904-05-05 12:34:56.1536,1904-12-25 -1904-05-05 12:34:56.1537,1904-12-25 -1904-05-05 12:34:56.1538,1904-12-25 -1904-05-05 12:34:56.1539,1904-12-25 -1904-05-05 12:34:56.154,1904-12-25 -1904-05-05 12:34:56.1541,1904-12-25 -1904-05-05 12:34:56.1542,1904-12-25 -1904-05-05 12:34:56.1543,1904-12-25 -1904-05-05 12:34:56.1544,1904-12-25 -1904-05-05 12:34:56.1545,1904-12-25 -1904-05-05 12:34:56.1546,1904-12-25 -1904-05-05 12:34:56.1547,1904-12-25 -1904-05-05 12:34:56.1548,1904-12-25 -1904-05-05 12:34:56.1549,1904-12-25 -1904-05-05 12:34:56.155,1904-12-25 -1904-05-05 12:34:56.1551,1904-12-25 -1904-05-05 12:34:56.1552,1904-12-25 -1904-05-05 12:34:56.1553,1904-12-25 -1904-05-05 12:34:56.1554,1904-12-25 -1904-05-05 12:34:56.1555,1904-12-25 -1904-05-05 12:34:56.1556,1904-12-25 -1904-05-05 12:34:56.1557,1904-12-25 -1904-05-05 12:34:56.1558,1904-12-25 -1904-05-05 12:34:56.1559,1904-12-25 -1904-05-05 12:34:56.156,1904-12-25 -1904-05-05 12:34:56.1561,1904-12-25 -1904-05-05 12:34:56.1562,1904-12-25 -1904-05-05 12:34:56.1563,1904-12-25 -1904-05-05 12:34:56.1564,1904-12-25 -1904-05-05 12:34:56.1565,1904-12-25 -1904-05-05 12:34:56.1566,1904-12-25 -1904-05-05 12:34:56.1567,1904-12-25 -1904-05-05 12:34:56.1568,1904-12-25 -1904-05-05 12:34:56.1569,1904-12-25 -1904-05-05 12:34:56.157,1904-12-25 -1904-05-05 12:34:56.1571,1904-12-25 -1904-05-05 12:34:56.1572,1904-12-25 -1904-05-05 12:34:56.1573,1904-12-25 -1904-05-05 12:34:56.1574,1904-12-25 -1904-05-05 12:34:56.1575,1904-12-25 -1904-05-05 12:34:56.1576,1904-12-25 -1904-05-05 12:34:56.1577,1904-12-25 -1904-05-05 12:34:56.1578,1904-12-25 -1904-05-05 12:34:56.1579,1904-12-25 -1904-05-05 12:34:56.158,1904-12-25 -1904-05-05 12:34:56.1581,1904-12-25 -1904-05-05 12:34:56.1582,1904-12-25 -1904-05-05 12:34:56.1583,1904-12-25 -1904-05-05 12:34:56.1584,1904-12-25 -1904-05-05 12:34:56.1585,1904-12-25 -1904-05-05 12:34:56.1586,1904-12-25 -1904-05-05 12:34:56.1587,1904-12-25 -1904-05-05 12:34:56.1588,1904-12-25 -1904-05-05 12:34:56.1589,1904-12-25 -1904-05-05 12:34:56.159,1904-12-25 -1904-05-05 12:34:56.1591,1904-12-25 -1904-05-05 12:34:56.1592,1904-12-25 -1904-05-05 12:34:56.1593,1904-12-25 -1904-05-05 12:34:56.1594,1904-12-25 -1904-05-05 12:34:56.1595,1904-12-25 -1904-05-05 12:34:56.1596,1904-12-25 -1904-05-05 12:34:56.1597,1904-12-25 -1904-05-05 12:34:56.1598,1904-12-25 -1904-05-05 12:34:56.1599,1904-12-25 -1904-05-05 12:34:56.16,1904-12-25 -1904-05-05 12:34:56.1601,1904-12-25 -1904-05-05 12:34:56.1602,1904-12-25 -1904-05-05 12:34:56.1603,1904-12-25 -1904-05-05 12:34:56.1604,1904-12-25 -1904-05-05 12:34:56.1605,1904-12-25 -1904-05-05 12:34:56.1606,1904-12-25 -1904-05-05 12:34:56.1607,1904-12-25 -1904-05-05 12:34:56.1608,1904-12-25 -1904-05-05 12:34:56.1609,1904-12-25 -1904-05-05 12:34:56.161,1904-12-25 -1904-05-05 12:34:56.1611,1904-12-25 -1904-05-05 12:34:56.1612,1904-12-25 -1904-05-05 12:34:56.1613,1904-12-25 -1904-05-05 12:34:56.1614,1904-12-25 -1904-05-05 12:34:56.1615,1904-12-25 -1904-05-05 12:34:56.1616,1904-12-25 -1904-05-05 12:34:56.1617,1904-12-25 -1904-05-05 12:34:56.1618,1904-12-25 -1904-05-05 12:34:56.1619,1904-12-25 -1904-05-05 12:34:56.162,1904-12-25 -1904-05-05 12:34:56.1621,1904-12-25 -1904-05-05 12:34:56.1622,1904-12-25 -1904-05-05 12:34:56.1623,1904-12-25 -1904-05-05 12:34:56.1624,1904-12-25 -1904-05-05 12:34:56.1625,1904-12-25 -1904-05-05 12:34:56.1626,1904-12-25 -1904-05-05 12:34:56.1627,1904-12-25 -1904-05-05 12:34:56.1628,1904-12-25 -1904-05-05 12:34:56.1629,1904-12-25 -1904-05-05 12:34:56.163,1904-12-25 -1904-05-05 12:34:56.1631,1904-12-25 -1904-05-05 12:34:56.1632,1904-12-25 -1904-05-05 12:34:56.1633,1904-12-25 -1904-05-05 12:34:56.1634,1904-12-25 -1904-05-05 12:34:56.1635,1904-12-25 -1904-05-05 12:34:56.1636,1904-12-25 -1904-05-05 12:34:56.1637,1904-12-25 -1904-05-05 12:34:56.1638,1904-12-25 -1904-05-05 12:34:56.1639,1904-12-25 -1904-05-05 12:34:56.164,1904-12-25 -1904-05-05 12:34:56.1641,1904-12-25 -1904-05-05 12:34:56.1642,1904-12-25 -1904-05-05 12:34:56.1643,1904-12-25 -1904-05-05 12:34:56.1644,1904-12-25 -1904-05-05 12:34:56.1645,1904-12-25 -1904-05-05 12:34:56.1646,1904-12-25 -1904-05-05 12:34:56.1647,1904-12-25 -1904-05-05 12:34:56.1648,1904-12-25 -1904-05-05 12:34:56.1649,1904-12-25 -1904-05-05 12:34:56.165,1904-12-25 -1904-05-05 12:34:56.1651,1904-12-25 -1904-05-05 12:34:56.1652,1904-12-25 -1904-05-05 12:34:56.1653,1904-12-25 -1904-05-05 12:34:56.1654,1904-12-25 -1904-05-05 12:34:56.1655,1904-12-25 -1904-05-05 12:34:56.1656,1904-12-25 -1904-05-05 12:34:56.1657,1904-12-25 -1904-05-05 12:34:56.1658,1904-12-25 -1904-05-05 12:34:56.1659,1904-12-25 -1904-05-05 12:34:56.166,1904-12-25 -1904-05-05 12:34:56.1661,1904-12-25 -1904-05-05 12:34:56.1662,1904-12-25 -1904-05-05 12:34:56.1663,1904-12-25 -1904-05-05 12:34:56.1664,1904-12-25 -1904-05-05 12:34:56.1665,1904-12-25 -1904-05-05 12:34:56.1666,1904-12-25 -1904-05-05 12:34:56.1667,1904-12-25 -1904-05-05 12:34:56.1668,1904-12-25 -1904-05-05 12:34:56.1669,1904-12-25 -1904-05-05 12:34:56.167,1904-12-25 -1904-05-05 12:34:56.1671,1904-12-25 -1904-05-05 12:34:56.1672,1904-12-25 -1904-05-05 12:34:56.1673,1904-12-25 -1904-05-05 12:34:56.1674,1904-12-25 -1904-05-05 12:34:56.1675,1904-12-25 -1904-05-05 12:34:56.1676,1904-12-25 -1904-05-05 12:34:56.1677,1904-12-25 -1904-05-05 12:34:56.1678,1904-12-25 -1904-05-05 12:34:56.1679,1904-12-25 -1904-05-05 12:34:56.168,1904-12-25 -1904-05-05 12:34:56.1681,1904-12-25 -1904-05-05 12:34:56.1682,1904-12-25 -1904-05-05 12:34:56.1683,1904-12-25 -1904-05-05 12:34:56.1684,1904-12-25 -1904-05-05 12:34:56.1685,1904-12-25 -1904-05-05 12:34:56.1686,1904-12-25 -1904-05-05 12:34:56.1687,1904-12-25 -1904-05-05 12:34:56.1688,1904-12-25 -1904-05-05 12:34:56.1689,1904-12-25 -1904-05-05 12:34:56.169,1904-12-25 -1904-05-05 12:34:56.1691,1904-12-25 -1904-05-05 12:34:56.1692,1904-12-25 -1904-05-05 12:34:56.1693,1904-12-25 -1904-05-05 12:34:56.1694,1904-12-25 -1904-05-05 12:34:56.1695,1904-12-25 -1904-05-05 12:34:56.1696,1904-12-25 -1904-05-05 12:34:56.1697,1904-12-25 -1904-05-05 12:34:56.1698,1904-12-25 -1904-05-05 12:34:56.1699,1904-12-25 -1904-05-05 12:34:56.17,1904-12-25 -1904-05-05 12:34:56.1701,1904-12-25 -1904-05-05 12:34:56.1702,1904-12-25 -1904-05-05 12:34:56.1703,1904-12-25 -1904-05-05 12:34:56.1704,1904-12-25 -1904-05-05 12:34:56.1705,1904-12-25 -1904-05-05 12:34:56.1706,1904-12-25 -1904-05-05 12:34:56.1707,1904-12-25 -1904-05-05 12:34:56.1708,1904-12-25 -1904-05-05 12:34:56.1709,1904-12-25 -1904-05-05 12:34:56.171,1904-12-25 -1904-05-05 12:34:56.1711,1904-12-25 -1904-05-05 12:34:56.1712,1904-12-25 -1904-05-05 12:34:56.1713,1904-12-25 -1904-05-05 12:34:56.1714,1904-12-25 -1904-05-05 12:34:56.1715,1904-12-25 -1904-05-05 12:34:56.1716,1904-12-25 -1904-05-05 12:34:56.1717,1904-12-25 -1904-05-05 12:34:56.1718,1904-12-25 -1904-05-05 12:34:56.1719,1904-12-25 -1904-05-05 12:34:56.172,1904-12-25 -1904-05-05 12:34:56.1721,1904-12-25 -1904-05-05 12:34:56.1722,1904-12-25 -1904-05-05 12:34:56.1723,1904-12-25 -1904-05-05 12:34:56.1724,1904-12-25 -1904-05-05 12:34:56.1725,1904-12-25 -1904-05-05 12:34:56.1726,1904-12-25 -1904-05-05 12:34:56.1727,1904-12-25 -1904-05-05 12:34:56.1728,1904-12-25 -1904-05-05 12:34:56.1729,1904-12-25 -1904-05-05 12:34:56.173,1904-12-25 -1904-05-05 12:34:56.1731,1904-12-25 -1904-05-05 12:34:56.1732,1904-12-25 -1904-05-05 12:34:56.1733,1904-12-25 -1904-05-05 12:34:56.1734,1904-12-25 -1904-05-05 12:34:56.1735,1904-12-25 -1904-05-05 12:34:56.1736,1904-12-25 -1904-05-05 12:34:56.1737,1904-12-25 -1904-05-05 12:34:56.1738,1904-12-25 -1904-05-05 12:34:56.1739,1904-12-25 -1904-05-05 12:34:56.174,1904-12-25 -1904-05-05 12:34:56.1741,1904-12-25 -1904-05-05 12:34:56.1742,1904-12-25 -1904-05-05 12:34:56.1743,1904-12-25 -1904-05-05 12:34:56.1744,1904-12-25 -1904-05-05 12:34:56.1745,1904-12-25 -1904-05-05 12:34:56.1746,1904-12-25 -1904-05-05 12:34:56.1747,1904-12-25 -1904-05-05 12:34:56.1748,1904-12-25 -1904-05-05 12:34:56.1749,1904-12-25 -1904-05-05 12:34:56.175,1904-12-25 -1904-05-05 12:34:56.1751,1904-12-25 -1904-05-05 12:34:56.1752,1904-12-25 -1904-05-05 12:34:56.1753,1904-12-25 -1904-05-05 12:34:56.1754,1904-12-25 -1904-05-05 12:34:56.1755,1904-12-25 -1904-05-05 12:34:56.1756,1904-12-25 -1904-05-05 12:34:56.1757,1904-12-25 -1904-05-05 12:34:56.1758,1904-12-25 -1904-05-05 12:34:56.1759,1904-12-25 -1904-05-05 12:34:56.176,1904-12-25 -1904-05-05 12:34:56.1761,1904-12-25 -1904-05-05 12:34:56.1762,1904-12-25 -1904-05-05 12:34:56.1763,1904-12-25 -1904-05-05 12:34:56.1764,1904-12-25 -1904-05-05 12:34:56.1765,1904-12-25 -1904-05-05 12:34:56.1766,1904-12-25 -1904-05-05 12:34:56.1767,1904-12-25 -1904-05-05 12:34:56.1768,1904-12-25 -1904-05-05 12:34:56.1769,1904-12-25 -1904-05-05 12:34:56.177,1904-12-25 -1904-05-05 12:34:56.1771,1904-12-25 -1904-05-05 12:34:56.1772,1904-12-25 -1904-05-05 12:34:56.1773,1904-12-25 -1904-05-05 12:34:56.1774,1904-12-25 -1904-05-05 12:34:56.1775,1904-12-25 -1904-05-05 12:34:56.1776,1904-12-25 -1904-05-05 12:34:56.1777,1904-12-25 -1904-05-05 12:34:56.1778,1904-12-25 -1904-05-05 12:34:56.1779,1904-12-25 -1904-05-05 12:34:56.178,1904-12-25 -1904-05-05 12:34:56.1781,1904-12-25 -1904-05-05 12:34:56.1782,1904-12-25 -1904-05-05 12:34:56.1783,1904-12-25 -1904-05-05 12:34:56.1784,1904-12-25 -1904-05-05 12:34:56.1785,1904-12-25 -1904-05-05 12:34:56.1786,1904-12-25 -1904-05-05 12:34:56.1787,1904-12-25 -1904-05-05 12:34:56.1788,1904-12-25 -1904-05-05 12:34:56.1789,1904-12-25 -1904-05-05 12:34:56.179,1904-12-25 -1904-05-05 12:34:56.1791,1904-12-25 -1904-05-05 12:34:56.1792,1904-12-25 -1904-05-05 12:34:56.1793,1904-12-25 -1904-05-05 12:34:56.1794,1904-12-25 -1904-05-05 12:34:56.1795,1904-12-25 -1904-05-05 12:34:56.1796,1904-12-25 -1904-05-05 12:34:56.1797,1904-12-25 -1904-05-05 12:34:56.1798,1904-12-25 -1904-05-05 12:34:56.1799,1904-12-25 -1904-05-05 12:34:56.18,1904-12-25 -1904-05-05 12:34:56.1801,1904-12-25 -1904-05-05 12:34:56.1802,1904-12-25 -1904-05-05 12:34:56.1803,1904-12-25 -1904-05-05 12:34:56.1804,1904-12-25 -1904-05-05 12:34:56.1805,1904-12-25 -1904-05-05 12:34:56.1806,1904-12-25 -1904-05-05 12:34:56.1807,1904-12-25 -1904-05-05 12:34:56.1808,1904-12-25 -1904-05-05 12:34:56.1809,1904-12-25 -1904-05-05 12:34:56.181,1904-12-25 -1904-05-05 12:34:56.1811,1904-12-25 -1904-05-05 12:34:56.1812,1904-12-25 -1904-05-05 12:34:56.1813,1904-12-25 -1904-05-05 12:34:56.1814,1904-12-25 -1904-05-05 12:34:56.1815,1904-12-25 -1904-05-05 12:34:56.1816,1904-12-25 -1904-05-05 12:34:56.1817,1904-12-25 -1904-05-05 12:34:56.1818,1904-12-25 -1904-05-05 12:34:56.1819,1904-12-25 -1904-05-05 12:34:56.182,1904-12-25 -1904-05-05 12:34:56.1821,1904-12-25 -1904-05-05 12:34:56.1822,1904-12-25 -1904-05-05 12:34:56.1823,1904-12-25 -1904-05-05 12:34:56.1824,1904-12-25 -1904-05-05 12:34:56.1825,1904-12-25 -1904-05-05 12:34:56.1826,1904-12-25 -1904-05-05 12:34:56.1827,1904-12-25 -1904-05-05 12:34:56.1828,1904-12-25 -1904-05-05 12:34:56.1829,1904-12-25 -1904-05-05 12:34:56.183,1904-12-25 -1904-05-05 12:34:56.1831,1904-12-25 -1904-05-05 12:34:56.1832,1904-12-25 -1904-05-05 12:34:56.1833,1904-12-25 -1904-05-05 12:34:56.1834,1904-12-25 -1904-05-05 12:34:56.1835,1904-12-25 -1904-05-05 12:34:56.1836,1904-12-25 -1904-05-05 12:34:56.1837,1904-12-25 -1904-05-05 12:34:56.1838,1904-12-25 -1904-05-05 12:34:56.1839,1904-12-25 -1904-05-05 12:34:56.184,1904-12-25 -1904-05-05 12:34:56.1841,1904-12-25 -1904-05-05 12:34:56.1842,1904-12-25 -1904-05-05 12:34:56.1843,1904-12-25 -1904-05-05 12:34:56.1844,1904-12-25 -1904-05-05 12:34:56.1845,1904-12-25 -1904-05-05 12:34:56.1846,1904-12-25 -1904-05-05 12:34:56.1847,1904-12-25 -1904-05-05 12:34:56.1848,1904-12-25 -1904-05-05 12:34:56.1849,1904-12-25 -1904-05-05 12:34:56.185,1904-12-25 -1904-05-05 12:34:56.1851,1904-12-25 -1904-05-05 12:34:56.1852,1904-12-25 -1904-05-05 12:34:56.1853,1904-12-25 -1904-05-05 12:34:56.1854,1904-12-25 -1904-05-05 12:34:56.1855,1904-12-25 -1904-05-05 12:34:56.1856,1904-12-25 -1904-05-05 12:34:56.1857,1904-12-25 -1904-05-05 12:34:56.1858,1904-12-25 -1904-05-05 12:34:56.1859,1904-12-25 -1904-05-05 12:34:56.186,1904-12-25 -1904-05-05 12:34:56.1861,1904-12-25 -1904-05-05 12:34:56.1862,1904-12-25 -1904-05-05 12:34:56.1863,1904-12-25 -1904-05-05 12:34:56.1864,1904-12-25 -1904-05-05 12:34:56.1865,1904-12-25 -1904-05-05 12:34:56.1866,1904-12-25 -1904-05-05 12:34:56.1867,1904-12-25 -1904-05-05 12:34:56.1868,1904-12-25 -1904-05-05 12:34:56.1869,1904-12-25 -1904-05-05 12:34:56.187,1904-12-25 -1904-05-05 12:34:56.1871,1904-12-25 -1904-05-05 12:34:56.1872,1904-12-25 -1904-05-05 12:34:56.1873,1904-12-25 -1904-05-05 12:34:56.1874,1904-12-25 -1904-05-05 12:34:56.1875,1904-12-25 -1904-05-05 12:34:56.1876,1904-12-25 -1904-05-05 12:34:56.1877,1904-12-25 -1904-05-05 12:34:56.1878,1904-12-25 -1904-05-05 12:34:56.1879,1904-12-25 -1904-05-05 12:34:56.188,1904-12-25 -1904-05-05 12:34:56.1881,1904-12-25 -1904-05-05 12:34:56.1882,1904-12-25 -1904-05-05 12:34:56.1883,1904-12-25 -1904-05-05 12:34:56.1884,1904-12-25 -1904-05-05 12:34:56.1885,1904-12-25 -1904-05-05 12:34:56.1886,1904-12-25 -1904-05-05 12:34:56.1887,1904-12-25 -1904-05-05 12:34:56.1888,1904-12-25 -1904-05-05 12:34:56.1889,1904-12-25 -1904-05-05 12:34:56.189,1904-12-25 -1904-05-05 12:34:56.1891,1904-12-25 -1904-05-05 12:34:56.1892,1904-12-25 -1904-05-05 12:34:56.1893,1904-12-25 -1904-05-05 12:34:56.1894,1904-12-25 -1904-05-05 12:34:56.1895,1904-12-25 -1904-05-05 12:34:56.1896,1904-12-25 -1904-05-05 12:34:56.1897,1904-12-25 -1904-05-05 12:34:56.1898,1904-12-25 -1904-05-05 12:34:56.1899,1904-12-25 -1904-05-05 12:34:56.19,1904-12-25 -1904-05-05 12:34:56.1901,1904-12-25 -1904-05-05 12:34:56.1902,1904-12-25 -1904-05-05 12:34:56.1903,1904-12-25 -1904-05-05 12:34:56.1904,1904-12-25 -1904-05-05 12:34:56.1905,1904-12-25 -1904-05-05 12:34:56.1906,1904-12-25 -1904-05-05 12:34:56.1907,1904-12-25 -1904-05-05 12:34:56.1908,1904-12-25 -1904-05-05 12:34:56.1909,1904-12-25 -1904-05-05 12:34:56.191,1904-12-25 -1904-05-05 12:34:56.1911,1904-12-25 -1904-05-05 12:34:56.1912,1904-12-25 -1904-05-05 12:34:56.1913,1904-12-25 -1904-05-05 12:34:56.1914,1904-12-25 -1904-05-05 12:34:56.1915,1904-12-25 -1904-05-05 12:34:56.1916,1904-12-25 -1904-05-05 12:34:56.1917,1904-12-25 -1904-05-05 12:34:56.1918,1904-12-25 -1904-05-05 12:34:56.1919,1904-12-25 -1904-05-05 12:34:56.192,1904-12-25 -1904-05-05 12:34:56.1921,1904-12-25 -1904-05-05 12:34:56.1922,1904-12-25 -1904-05-05 12:34:56.1923,1904-12-25 -1904-05-05 12:34:56.1924,1904-12-25 -1904-05-05 12:34:56.1925,1904-12-25 -1904-05-05 12:34:56.1926,1904-12-25 -1904-05-05 12:34:56.1927,1904-12-25 -1904-05-05 12:34:56.1928,1904-12-25 -1904-05-05 12:34:56.1929,1904-12-25 -1904-05-05 12:34:56.193,1904-12-25 -1904-05-05 12:34:56.1931,1904-12-25 -1904-05-05 12:34:56.1932,1904-12-25 -1904-05-05 12:34:56.1933,1904-12-25 -1904-05-05 12:34:56.1934,1904-12-25 -1904-05-05 12:34:56.1935,1904-12-25 -1904-05-05 12:34:56.1936,1904-12-25 -1904-05-05 12:34:56.1937,1904-12-25 -1904-05-05 12:34:56.1938,1904-12-25 -1904-05-05 12:34:56.1939,1904-12-25 -1904-05-05 12:34:56.194,1904-12-25 -1904-05-05 12:34:56.1941,1904-12-25 -1904-05-05 12:34:56.1942,1904-12-25 -1904-05-05 12:34:56.1943,1904-12-25 -1904-05-05 12:34:56.1944,1904-12-25 -1904-05-05 12:34:56.1945,1904-12-25 -1904-05-05 12:34:56.1946,1904-12-25 -1904-05-05 12:34:56.1947,1904-12-25 -1904-05-05 12:34:56.1948,1904-12-25 -1904-05-05 12:34:56.1949,1904-12-25 -1904-05-05 12:34:56.195,1904-12-25 -1904-05-05 12:34:56.1951,1904-12-25 -1904-05-05 12:34:56.1952,1904-12-25 -1904-05-05 12:34:56.1953,1904-12-25 -1904-05-05 12:34:56.1954,1904-12-25 -1904-05-05 12:34:56.1955,1904-12-25 -1904-05-05 12:34:56.1956,1904-12-25 -1904-05-05 12:34:56.1957,1904-12-25 -1904-05-05 12:34:56.1958,1904-12-25 -1904-05-05 12:34:56.1959,1904-12-25 -1904-05-05 12:34:56.196,1904-12-25 -1904-05-05 12:34:56.1961,1904-12-25 -1904-05-05 12:34:56.1962,1904-12-25 -1904-05-05 12:34:56.1963,1904-12-25 -1904-05-05 12:34:56.1964,1904-12-25 -1904-05-05 12:34:56.1965,1904-12-25 -1904-05-05 12:34:56.1966,1904-12-25 -1904-05-05 12:34:56.1967,1904-12-25 -1904-05-05 12:34:56.1968,1904-12-25 -1904-05-05 12:34:56.1969,1904-12-25 -1904-05-05 12:34:56.197,1904-12-25 -1904-05-05 12:34:56.1971,1904-12-25 -1904-05-05 12:34:56.1972,1904-12-25 -1904-05-05 12:34:56.1973,1904-12-25 -1904-05-05 12:34:56.1974,1904-12-25 -1904-05-05 12:34:56.1975,1904-12-25 -1904-05-05 12:34:56.1976,1904-12-25 -1904-05-05 12:34:56.1977,1904-12-25 -1904-05-05 12:34:56.1978,1904-12-25 -1904-05-05 12:34:56.1979,1904-12-25 -1904-05-05 12:34:56.198,1904-12-25 -1904-05-05 12:34:56.1981,1904-12-25 -1904-05-05 12:34:56.1982,1904-12-25 -1904-05-05 12:34:56.1983,1904-12-25 -1904-05-05 12:34:56.1984,1904-12-25 -1904-05-05 12:34:56.1985,1904-12-25 -1904-05-05 12:34:56.1986,1904-12-25 -1904-05-05 12:34:56.1987,1904-12-25 -1904-05-05 12:34:56.1988,1904-12-25 -1904-05-05 12:34:56.1989,1904-12-25 -1904-05-05 12:34:56.199,1904-12-25 -1904-05-05 12:34:56.1991,1904-12-25 -1904-05-05 12:34:56.1992,1904-12-25 -1904-05-05 12:34:56.1993,1904-12-25 -1904-05-05 12:34:56.1994,1904-12-25 -1904-05-05 12:34:56.1995,1904-12-25 -1904-05-05 12:34:56.1996,1904-12-25 -1904-05-05 12:34:56.1997,1904-12-25 -1904-05-05 12:34:56.1998,1904-12-25 -1904-05-05 12:34:56.1999,1904-12-25 -1905-05-05 12:34:56.1,1905-12-25 -1905-05-05 12:34:56.1001,1905-12-25 -1905-05-05 12:34:56.1002,1905-12-25 -1905-05-05 12:34:56.1003,1905-12-25 -1905-05-05 12:34:56.1004,1905-12-25 -1905-05-05 12:34:56.1005,1905-12-25 -1905-05-05 12:34:56.1006,1905-12-25 -1905-05-05 12:34:56.1007,1905-12-25 -1905-05-05 12:34:56.1008,1905-12-25 -1905-05-05 12:34:56.1009,1905-12-25 -1905-05-05 12:34:56.101,1905-12-25 -1905-05-05 12:34:56.1011,1905-12-25 -1905-05-05 12:34:56.1012,1905-12-25 -1905-05-05 12:34:56.1013,1905-12-25 -1905-05-05 12:34:56.1014,1905-12-25 -1905-05-05 12:34:56.1015,1905-12-25 -1905-05-05 12:34:56.1016,1905-12-25 -1905-05-05 12:34:56.1017,1905-12-25 -1905-05-05 12:34:56.1018,1905-12-25 -1905-05-05 12:34:56.1019,1905-12-25 -1905-05-05 12:34:56.102,1905-12-25 -1905-05-05 12:34:56.1021,1905-12-25 -1905-05-05 12:34:56.1022,1905-12-25 -1905-05-05 12:34:56.1023,1905-12-25 -1905-05-05 12:34:56.1024,1905-12-25 -1905-05-05 12:34:56.1025,1905-12-25 -1905-05-05 12:34:56.1026,1905-12-25 -1905-05-05 12:34:56.1027,1905-12-25 -1905-05-05 12:34:56.1028,1905-12-25 -1905-05-05 12:34:56.1029,1905-12-25 -1905-05-05 12:34:56.103,1905-12-25 -1905-05-05 12:34:56.1031,1905-12-25 -1905-05-05 12:34:56.1032,1905-12-25 -1905-05-05 12:34:56.1033,1905-12-25 -1905-05-05 12:34:56.1034,1905-12-25 -1905-05-05 12:34:56.1035,1905-12-25 -1905-05-05 12:34:56.1036,1905-12-25 -1905-05-05 12:34:56.1037,1905-12-25 -1905-05-05 12:34:56.1038,1905-12-25 -1905-05-05 12:34:56.1039,1905-12-25 -1905-05-05 12:34:56.104,1905-12-25 -1905-05-05 12:34:56.1041,1905-12-25 -1905-05-05 12:34:56.1042,1905-12-25 -1905-05-05 12:34:56.1043,1905-12-25 -1905-05-05 12:34:56.1044,1905-12-25 -1905-05-05 12:34:56.1045,1905-12-25 -1905-05-05 12:34:56.1046,1905-12-25 -1905-05-05 12:34:56.1047,1905-12-25 -1905-05-05 12:34:56.1048,1905-12-25 -1905-05-05 12:34:56.1049,1905-12-25 -1905-05-05 12:34:56.105,1905-12-25 -1905-05-05 12:34:56.1051,1905-12-25 -1905-05-05 12:34:56.1052,1905-12-25 -1905-05-05 12:34:56.1053,1905-12-25 -1905-05-05 12:34:56.1054,1905-12-25 -1905-05-05 12:34:56.1055,1905-12-25 -1905-05-05 12:34:56.1056,1905-12-25 -1905-05-05 12:34:56.1057,1905-12-25 -1905-05-05 12:34:56.1058,1905-12-25 -1905-05-05 12:34:56.1059,1905-12-25 -1905-05-05 12:34:56.106,1905-12-25 -1905-05-05 12:34:56.1061,1905-12-25 -1905-05-05 12:34:56.1062,1905-12-25 -1905-05-05 12:34:56.1063,1905-12-25 -1905-05-05 12:34:56.1064,1905-12-25 -1905-05-05 12:34:56.1065,1905-12-25 -1905-05-05 12:34:56.1066,1905-12-25 -1905-05-05 12:34:56.1067,1905-12-25 -1905-05-05 12:34:56.1068,1905-12-25 -1905-05-05 12:34:56.1069,1905-12-25 -1905-05-05 12:34:56.107,1905-12-25 -1905-05-05 12:34:56.1071,1905-12-25 -1905-05-05 12:34:56.1072,1905-12-25 -1905-05-05 12:34:56.1073,1905-12-25 -1905-05-05 12:34:56.1074,1905-12-25 -1905-05-05 12:34:56.1075,1905-12-25 -1905-05-05 12:34:56.1076,1905-12-25 -1905-05-05 12:34:56.1077,1905-12-25 -1905-05-05 12:34:56.1078,1905-12-25 -1905-05-05 12:34:56.1079,1905-12-25 -1905-05-05 12:34:56.108,1905-12-25 -1905-05-05 12:34:56.1081,1905-12-25 -1905-05-05 12:34:56.1082,1905-12-25 -1905-05-05 12:34:56.1083,1905-12-25 -1905-05-05 12:34:56.1084,1905-12-25 -1905-05-05 12:34:56.1085,1905-12-25 -1905-05-05 12:34:56.1086,1905-12-25 -1905-05-05 12:34:56.1087,1905-12-25 -1905-05-05 12:34:56.1088,1905-12-25 -1905-05-05 12:34:56.1089,1905-12-25 -1905-05-05 12:34:56.109,1905-12-25 -1905-05-05 12:34:56.1091,1905-12-25 -1905-05-05 12:34:56.1092,1905-12-25 -1905-05-05 12:34:56.1093,1905-12-25 -1905-05-05 12:34:56.1094,1905-12-25 -1905-05-05 12:34:56.1095,1905-12-25 -1905-05-05 12:34:56.1096,1905-12-25 -1905-05-05 12:34:56.1097,1905-12-25 -1905-05-05 12:34:56.1098,1905-12-25 -1905-05-05 12:34:56.1099,1905-12-25 -1905-05-05 12:34:56.11,1905-12-25 -1905-05-05 12:34:56.1101,1905-12-25 -1905-05-05 12:34:56.1102,1905-12-25 -1905-05-05 12:34:56.1103,1905-12-25 -1905-05-05 12:34:56.1104,1905-12-25 -1905-05-05 12:34:56.1105,1905-12-25 -1905-05-05 12:34:56.1106,1905-12-25 -1905-05-05 12:34:56.1107,1905-12-25 -1905-05-05 12:34:56.1108,1905-12-25 -1905-05-05 12:34:56.1109,1905-12-25 -1905-05-05 12:34:56.111,1905-12-25 -1905-05-05 12:34:56.1111,1905-12-25 -1905-05-05 12:34:56.1112,1905-12-25 -1905-05-05 12:34:56.1113,1905-12-25 -1905-05-05 12:34:56.1114,1905-12-25 -1905-05-05 12:34:56.1115,1905-12-25 -1905-05-05 12:34:56.1116,1905-12-25 -1905-05-05 12:34:56.1117,1905-12-25 -1905-05-05 12:34:56.1118,1905-12-25 -1905-05-05 12:34:56.1119,1905-12-25 -1905-05-05 12:34:56.112,1905-12-25 -1905-05-05 12:34:56.1121,1905-12-25 -1905-05-05 12:34:56.1122,1905-12-25 -1905-05-05 12:34:56.1123,1905-12-25 -1905-05-05 12:34:56.1124,1905-12-25 -1905-05-05 12:34:56.1125,1905-12-25 -1905-05-05 12:34:56.1126,1905-12-25 -1905-05-05 12:34:56.1127,1905-12-25 -1905-05-05 12:34:56.1128,1905-12-25 -1905-05-05 12:34:56.1129,1905-12-25 -1905-05-05 12:34:56.113,1905-12-25 -1905-05-05 12:34:56.1131,1905-12-25 -1905-05-05 12:34:56.1132,1905-12-25 -1905-05-05 12:34:56.1133,1905-12-25 -1905-05-05 12:34:56.1134,1905-12-25 -1905-05-05 12:34:56.1135,1905-12-25 -1905-05-05 12:34:56.1136,1905-12-25 -1905-05-05 12:34:56.1137,1905-12-25 -1905-05-05 12:34:56.1138,1905-12-25 -1905-05-05 12:34:56.1139,1905-12-25 -1905-05-05 12:34:56.114,1905-12-25 -1905-05-05 12:34:56.1141,1905-12-25 -1905-05-05 12:34:56.1142,1905-12-25 -1905-05-05 12:34:56.1143,1905-12-25 -1905-05-05 12:34:56.1144,1905-12-25 -1905-05-05 12:34:56.1145,1905-12-25 -1905-05-05 12:34:56.1146,1905-12-25 -1905-05-05 12:34:56.1147,1905-12-25 -1905-05-05 12:34:56.1148,1905-12-25 -1905-05-05 12:34:56.1149,1905-12-25 -1905-05-05 12:34:56.115,1905-12-25 -1905-05-05 12:34:56.1151,1905-12-25 -1905-05-05 12:34:56.1152,1905-12-25 -1905-05-05 12:34:56.1153,1905-12-25 -1905-05-05 12:34:56.1154,1905-12-25 -1905-05-05 12:34:56.1155,1905-12-25 -1905-05-05 12:34:56.1156,1905-12-25 -1905-05-05 12:34:56.1157,1905-12-25 -1905-05-05 12:34:56.1158,1905-12-25 -1905-05-05 12:34:56.1159,1905-12-25 -1905-05-05 12:34:56.116,1905-12-25 -1905-05-05 12:34:56.1161,1905-12-25 -1905-05-05 12:34:56.1162,1905-12-25 -1905-05-05 12:34:56.1163,1905-12-25 -1905-05-05 12:34:56.1164,1905-12-25 -1905-05-05 12:34:56.1165,1905-12-25 -1905-05-05 12:34:56.1166,1905-12-25 -1905-05-05 12:34:56.1167,1905-12-25 -1905-05-05 12:34:56.1168,1905-12-25 -1905-05-05 12:34:56.1169,1905-12-25 -1905-05-05 12:34:56.117,1905-12-25 -1905-05-05 12:34:56.1171,1905-12-25 -1905-05-05 12:34:56.1172,1905-12-25 -1905-05-05 12:34:56.1173,1905-12-25 -1905-05-05 12:34:56.1174,1905-12-25 -1905-05-05 12:34:56.1175,1905-12-25 -1905-05-05 12:34:56.1176,1905-12-25 -1905-05-05 12:34:56.1177,1905-12-25 -1905-05-05 12:34:56.1178,1905-12-25 -1905-05-05 12:34:56.1179,1905-12-25 -1905-05-05 12:34:56.118,1905-12-25 -1905-05-05 12:34:56.1181,1905-12-25 -1905-05-05 12:34:56.1182,1905-12-25 -1905-05-05 12:34:56.1183,1905-12-25 -1905-05-05 12:34:56.1184,1905-12-25 -1905-05-05 12:34:56.1185,1905-12-25 -1905-05-05 12:34:56.1186,1905-12-25 -1905-05-05 12:34:56.1187,1905-12-25 -1905-05-05 12:34:56.1188,1905-12-25 -1905-05-05 12:34:56.1189,1905-12-25 -1905-05-05 12:34:56.119,1905-12-25 -1905-05-05 12:34:56.1191,1905-12-25 -1905-05-05 12:34:56.1192,1905-12-25 -1905-05-05 12:34:56.1193,1905-12-25 -1905-05-05 12:34:56.1194,1905-12-25 -1905-05-05 12:34:56.1195,1905-12-25 -1905-05-05 12:34:56.1196,1905-12-25 -1905-05-05 12:34:56.1197,1905-12-25 -1905-05-05 12:34:56.1198,1905-12-25 -1905-05-05 12:34:56.1199,1905-12-25 -1905-05-05 12:34:56.12,1905-12-25 -1905-05-05 12:34:56.1201,1905-12-25 -1905-05-05 12:34:56.1202,1905-12-25 -1905-05-05 12:34:56.1203,1905-12-25 -1905-05-05 12:34:56.1204,1905-12-25 -1905-05-05 12:34:56.1205,1905-12-25 -1905-05-05 12:34:56.1206,1905-12-25 -1905-05-05 12:34:56.1207,1905-12-25 -1905-05-05 12:34:56.1208,1905-12-25 -1905-05-05 12:34:56.1209,1905-12-25 -1905-05-05 12:34:56.121,1905-12-25 -1905-05-05 12:34:56.1211,1905-12-25 -1905-05-05 12:34:56.1212,1905-12-25 -1905-05-05 12:34:56.1213,1905-12-25 -1905-05-05 12:34:56.1214,1905-12-25 -1905-05-05 12:34:56.1215,1905-12-25 -1905-05-05 12:34:56.1216,1905-12-25 -1905-05-05 12:34:56.1217,1905-12-25 -1905-05-05 12:34:56.1218,1905-12-25 -1905-05-05 12:34:56.1219,1905-12-25 -1905-05-05 12:34:56.122,1905-12-25 -1905-05-05 12:34:56.1221,1905-12-25 -1905-05-05 12:34:56.1222,1905-12-25 -1905-05-05 12:34:56.1223,1905-12-25 -1905-05-05 12:34:56.1224,1905-12-25 -1905-05-05 12:34:56.1225,1905-12-25 -1905-05-05 12:34:56.1226,1905-12-25 -1905-05-05 12:34:56.1227,1905-12-25 -1905-05-05 12:34:56.1228,1905-12-25 -1905-05-05 12:34:56.1229,1905-12-25 -1905-05-05 12:34:56.123,1905-12-25 -1905-05-05 12:34:56.1231,1905-12-25 -1905-05-05 12:34:56.1232,1905-12-25 -1905-05-05 12:34:56.1233,1905-12-25 -1905-05-05 12:34:56.1234,1905-12-25 -1905-05-05 12:34:56.1235,1905-12-25 -1905-05-05 12:34:56.1236,1905-12-25 -1905-05-05 12:34:56.1237,1905-12-25 -1905-05-05 12:34:56.1238,1905-12-25 -1905-05-05 12:34:56.1239,1905-12-25 -1905-05-05 12:34:56.124,1905-12-25 -1905-05-05 12:34:56.1241,1905-12-25 -1905-05-05 12:34:56.1242,1905-12-25 -1905-05-05 12:34:56.1243,1905-12-25 -1905-05-05 12:34:56.1244,1905-12-25 -1905-05-05 12:34:56.1245,1905-12-25 -1905-05-05 12:34:56.1246,1905-12-25 -1905-05-05 12:34:56.1247,1905-12-25 -1905-05-05 12:34:56.1248,1905-12-25 -1905-05-05 12:34:56.1249,1905-12-25 -1905-05-05 12:34:56.125,1905-12-25 -1905-05-05 12:34:56.1251,1905-12-25 -1905-05-05 12:34:56.1252,1905-12-25 -1905-05-05 12:34:56.1253,1905-12-25 -1905-05-05 12:34:56.1254,1905-12-25 -1905-05-05 12:34:56.1255,1905-12-25 -1905-05-05 12:34:56.1256,1905-12-25 -1905-05-05 12:34:56.1257,1905-12-25 -1905-05-05 12:34:56.1258,1905-12-25 -1905-05-05 12:34:56.1259,1905-12-25 -1905-05-05 12:34:56.126,1905-12-25 -1905-05-05 12:34:56.1261,1905-12-25 -1905-05-05 12:34:56.1262,1905-12-25 -1905-05-05 12:34:56.1263,1905-12-25 -1905-05-05 12:34:56.1264,1905-12-25 -1905-05-05 12:34:56.1265,1905-12-25 -1905-05-05 12:34:56.1266,1905-12-25 -1905-05-05 12:34:56.1267,1905-12-25 -1905-05-05 12:34:56.1268,1905-12-25 -1905-05-05 12:34:56.1269,1905-12-25 -1905-05-05 12:34:56.127,1905-12-25 -1905-05-05 12:34:56.1271,1905-12-25 -1905-05-05 12:34:56.1272,1905-12-25 -1905-05-05 12:34:56.1273,1905-12-25 -1905-05-05 12:34:56.1274,1905-12-25 -1905-05-05 12:34:56.1275,1905-12-25 -1905-05-05 12:34:56.1276,1905-12-25 -1905-05-05 12:34:56.1277,1905-12-25 -1905-05-05 12:34:56.1278,1905-12-25 -1905-05-05 12:34:56.1279,1905-12-25 -1905-05-05 12:34:56.128,1905-12-25 -1905-05-05 12:34:56.1281,1905-12-25 -1905-05-05 12:34:56.1282,1905-12-25 -1905-05-05 12:34:56.1283,1905-12-25 -1905-05-05 12:34:56.1284,1905-12-25 -1905-05-05 12:34:56.1285,1905-12-25 -1905-05-05 12:34:56.1286,1905-12-25 -1905-05-05 12:34:56.1287,1905-12-25 -1905-05-05 12:34:56.1288,1905-12-25 -1905-05-05 12:34:56.1289,1905-12-25 -1905-05-05 12:34:56.129,1905-12-25 -1905-05-05 12:34:56.1291,1905-12-25 -1905-05-05 12:34:56.1292,1905-12-25 -1905-05-05 12:34:56.1293,1905-12-25 -1905-05-05 12:34:56.1294,1905-12-25 -1905-05-05 12:34:56.1295,1905-12-25 -1905-05-05 12:34:56.1296,1905-12-25 -1905-05-05 12:34:56.1297,1905-12-25 -1905-05-05 12:34:56.1298,1905-12-25 -1905-05-05 12:34:56.1299,1905-12-25 -1905-05-05 12:34:56.13,1905-12-25 -1905-05-05 12:34:56.1301,1905-12-25 -1905-05-05 12:34:56.1302,1905-12-25 -1905-05-05 12:34:56.1303,1905-12-25 -1905-05-05 12:34:56.1304,1905-12-25 -1905-05-05 12:34:56.1305,1905-12-25 -1905-05-05 12:34:56.1306,1905-12-25 -1905-05-05 12:34:56.1307,1905-12-25 -1905-05-05 12:34:56.1308,1905-12-25 -1905-05-05 12:34:56.1309,1905-12-25 -1905-05-05 12:34:56.131,1905-12-25 -1905-05-05 12:34:56.1311,1905-12-25 -1905-05-05 12:34:56.1312,1905-12-25 -1905-05-05 12:34:56.1313,1905-12-25 -1905-05-05 12:34:56.1314,1905-12-25 -1905-05-05 12:34:56.1315,1905-12-25 -1905-05-05 12:34:56.1316,1905-12-25 -1905-05-05 12:34:56.1317,1905-12-25 -1905-05-05 12:34:56.1318,1905-12-25 -1905-05-05 12:34:56.1319,1905-12-25 -1905-05-05 12:34:56.132,1905-12-25 -1905-05-05 12:34:56.1321,1905-12-25 -1905-05-05 12:34:56.1322,1905-12-25 -1905-05-05 12:34:56.1323,1905-12-25 -1905-05-05 12:34:56.1324,1905-12-25 -1905-05-05 12:34:56.1325,1905-12-25 -1905-05-05 12:34:56.1326,1905-12-25 -1905-05-05 12:34:56.1327,1905-12-25 -1905-05-05 12:34:56.1328,1905-12-25 -1905-05-05 12:34:56.1329,1905-12-25 -1905-05-05 12:34:56.133,1905-12-25 -1905-05-05 12:34:56.1331,1905-12-25 -1905-05-05 12:34:56.1332,1905-12-25 -1905-05-05 12:34:56.1333,1905-12-25 -1905-05-05 12:34:56.1334,1905-12-25 -1905-05-05 12:34:56.1335,1905-12-25 -1905-05-05 12:34:56.1336,1905-12-25 -1905-05-05 12:34:56.1337,1905-12-25 -1905-05-05 12:34:56.1338,1905-12-25 -1905-05-05 12:34:56.1339,1905-12-25 -1905-05-05 12:34:56.134,1905-12-25 -1905-05-05 12:34:56.1341,1905-12-25 -1905-05-05 12:34:56.1342,1905-12-25 -1905-05-05 12:34:56.1343,1905-12-25 -1905-05-05 12:34:56.1344,1905-12-25 -1905-05-05 12:34:56.1345,1905-12-25 -1905-05-05 12:34:56.1346,1905-12-25 -1905-05-05 12:34:56.1347,1905-12-25 -1905-05-05 12:34:56.1348,1905-12-25 -1905-05-05 12:34:56.1349,1905-12-25 -1905-05-05 12:34:56.135,1905-12-25 -1905-05-05 12:34:56.1351,1905-12-25 -1905-05-05 12:34:56.1352,1905-12-25 -1905-05-05 12:34:56.1353,1905-12-25 -1905-05-05 12:34:56.1354,1905-12-25 -1905-05-05 12:34:56.1355,1905-12-25 -1905-05-05 12:34:56.1356,1905-12-25 -1905-05-05 12:34:56.1357,1905-12-25 -1905-05-05 12:34:56.1358,1905-12-25 -1905-05-05 12:34:56.1359,1905-12-25 -1905-05-05 12:34:56.136,1905-12-25 -1905-05-05 12:34:56.1361,1905-12-25 -1905-05-05 12:34:56.1362,1905-12-25 -1905-05-05 12:34:56.1363,1905-12-25 -1905-05-05 12:34:56.1364,1905-12-25 -1905-05-05 12:34:56.1365,1905-12-25 -1905-05-05 12:34:56.1366,1905-12-25 -1905-05-05 12:34:56.1367,1905-12-25 -1905-05-05 12:34:56.1368,1905-12-25 -1905-05-05 12:34:56.1369,1905-12-25 -1905-05-05 12:34:56.137,1905-12-25 -1905-05-05 12:34:56.1371,1905-12-25 -1905-05-05 12:34:56.1372,1905-12-25 -1905-05-05 12:34:56.1373,1905-12-25 -1905-05-05 12:34:56.1374,1905-12-25 -1905-05-05 12:34:56.1375,1905-12-25 -1905-05-05 12:34:56.1376,1905-12-25 -1905-05-05 12:34:56.1377,1905-12-25 -1905-05-05 12:34:56.1378,1905-12-25 -1905-05-05 12:34:56.1379,1905-12-25 -1905-05-05 12:34:56.138,1905-12-25 -1905-05-05 12:34:56.1381,1905-12-25 -1905-05-05 12:34:56.1382,1905-12-25 -1905-05-05 12:34:56.1383,1905-12-25 -1905-05-05 12:34:56.1384,1905-12-25 -1905-05-05 12:34:56.1385,1905-12-25 -1905-05-05 12:34:56.1386,1905-12-25 -1905-05-05 12:34:56.1387,1905-12-25 -1905-05-05 12:34:56.1388,1905-12-25 -1905-05-05 12:34:56.1389,1905-12-25 -1905-05-05 12:34:56.139,1905-12-25 -1905-05-05 12:34:56.1391,1905-12-25 -1905-05-05 12:34:56.1392,1905-12-25 -1905-05-05 12:34:56.1393,1905-12-25 -1905-05-05 12:34:56.1394,1905-12-25 -1905-05-05 12:34:56.1395,1905-12-25 -1905-05-05 12:34:56.1396,1905-12-25 -1905-05-05 12:34:56.1397,1905-12-25 -1905-05-05 12:34:56.1398,1905-12-25 -1905-05-05 12:34:56.1399,1905-12-25 -1905-05-05 12:34:56.14,1905-12-25 -1905-05-05 12:34:56.1401,1905-12-25 -1905-05-05 12:34:56.1402,1905-12-25 -1905-05-05 12:34:56.1403,1905-12-25 -1905-05-05 12:34:56.1404,1905-12-25 -1905-05-05 12:34:56.1405,1905-12-25 -1905-05-05 12:34:56.1406,1905-12-25 -1905-05-05 12:34:56.1407,1905-12-25 -1905-05-05 12:34:56.1408,1905-12-25 -1905-05-05 12:34:56.1409,1905-12-25 -1905-05-05 12:34:56.141,1905-12-25 -1905-05-05 12:34:56.1411,1905-12-25 -1905-05-05 12:34:56.1412,1905-12-25 -1905-05-05 12:34:56.1413,1905-12-25 -1905-05-05 12:34:56.1414,1905-12-25 -1905-05-05 12:34:56.1415,1905-12-25 -1905-05-05 12:34:56.1416,1905-12-25 -1905-05-05 12:34:56.1417,1905-12-25 -1905-05-05 12:34:56.1418,1905-12-25 -1905-05-05 12:34:56.1419,1905-12-25 -1905-05-05 12:34:56.142,1905-12-25 -1905-05-05 12:34:56.1421,1905-12-25 -1905-05-05 12:34:56.1422,1905-12-25 -1905-05-05 12:34:56.1423,1905-12-25 -1905-05-05 12:34:56.1424,1905-12-25 -1905-05-05 12:34:56.1425,1905-12-25 -1905-05-05 12:34:56.1426,1905-12-25 -1905-05-05 12:34:56.1427,1905-12-25 -1905-05-05 12:34:56.1428,1905-12-25 -1905-05-05 12:34:56.1429,1905-12-25 -1905-05-05 12:34:56.143,1905-12-25 -1905-05-05 12:34:56.1431,1905-12-25 -1905-05-05 12:34:56.1432,1905-12-25 -1905-05-05 12:34:56.1433,1905-12-25 -1905-05-05 12:34:56.1434,1905-12-25 -1905-05-05 12:34:56.1435,1905-12-25 -1905-05-05 12:34:56.1436,1905-12-25 -1905-05-05 12:34:56.1437,1905-12-25 -1905-05-05 12:34:56.1438,1905-12-25 -1905-05-05 12:34:56.1439,1905-12-25 -1905-05-05 12:34:56.144,1905-12-25 -1905-05-05 12:34:56.1441,1905-12-25 -1905-05-05 12:34:56.1442,1905-12-25 -1905-05-05 12:34:56.1443,1905-12-25 -1905-05-05 12:34:56.1444,1905-12-25 -1905-05-05 12:34:56.1445,1905-12-25 -1905-05-05 12:34:56.1446,1905-12-25 -1905-05-05 12:34:56.1447,1905-12-25 -1905-05-05 12:34:56.1448,1905-12-25 -1905-05-05 12:34:56.1449,1905-12-25 -1905-05-05 12:34:56.145,1905-12-25 -1905-05-05 12:34:56.1451,1905-12-25 -1905-05-05 12:34:56.1452,1905-12-25 -1905-05-05 12:34:56.1453,1905-12-25 -1905-05-05 12:34:56.1454,1905-12-25 -1905-05-05 12:34:56.1455,1905-12-25 -1905-05-05 12:34:56.1456,1905-12-25 -1905-05-05 12:34:56.1457,1905-12-25 -1905-05-05 12:34:56.1458,1905-12-25 -1905-05-05 12:34:56.1459,1905-12-25 -1905-05-05 12:34:56.146,1905-12-25 -1905-05-05 12:34:56.1461,1905-12-25 -1905-05-05 12:34:56.1462,1905-12-25 -1905-05-05 12:34:56.1463,1905-12-25 -1905-05-05 12:34:56.1464,1905-12-25 -1905-05-05 12:34:56.1465,1905-12-25 -1905-05-05 12:34:56.1466,1905-12-25 -1905-05-05 12:34:56.1467,1905-12-25 -1905-05-05 12:34:56.1468,1905-12-25 -1905-05-05 12:34:56.1469,1905-12-25 -1905-05-05 12:34:56.147,1905-12-25 -1905-05-05 12:34:56.1471,1905-12-25 -1905-05-05 12:34:56.1472,1905-12-25 -1905-05-05 12:34:56.1473,1905-12-25 -1905-05-05 12:34:56.1474,1905-12-25 -1905-05-05 12:34:56.1475,1905-12-25 -1905-05-05 12:34:56.1476,1905-12-25 -1905-05-05 12:34:56.1477,1905-12-25 -1905-05-05 12:34:56.1478,1905-12-25 -1905-05-05 12:34:56.1479,1905-12-25 -1905-05-05 12:34:56.148,1905-12-25 -1905-05-05 12:34:56.1481,1905-12-25 -1905-05-05 12:34:56.1482,1905-12-25 -1905-05-05 12:34:56.1483,1905-12-25 -1905-05-05 12:34:56.1484,1905-12-25 -1905-05-05 12:34:56.1485,1905-12-25 -1905-05-05 12:34:56.1486,1905-12-25 -1905-05-05 12:34:56.1487,1905-12-25 -1905-05-05 12:34:56.1488,1905-12-25 -1905-05-05 12:34:56.1489,1905-12-25 -1905-05-05 12:34:56.149,1905-12-25 -1905-05-05 12:34:56.1491,1905-12-25 -1905-05-05 12:34:56.1492,1905-12-25 -1905-05-05 12:34:56.1493,1905-12-25 -1905-05-05 12:34:56.1494,1905-12-25 -1905-05-05 12:34:56.1495,1905-12-25 -1905-05-05 12:34:56.1496,1905-12-25 -1905-05-05 12:34:56.1497,1905-12-25 -1905-05-05 12:34:56.1498,1905-12-25 -1905-05-05 12:34:56.1499,1905-12-25 -1905-05-05 12:34:56.15,1905-12-25 -1905-05-05 12:34:56.1501,1905-12-25 -1905-05-05 12:34:56.1502,1905-12-25 -1905-05-05 12:34:56.1503,1905-12-25 -1905-05-05 12:34:56.1504,1905-12-25 -1905-05-05 12:34:56.1505,1905-12-25 -1905-05-05 12:34:56.1506,1905-12-25 -1905-05-05 12:34:56.1507,1905-12-25 -1905-05-05 12:34:56.1508,1905-12-25 -1905-05-05 12:34:56.1509,1905-12-25 -1905-05-05 12:34:56.151,1905-12-25 -1905-05-05 12:34:56.1511,1905-12-25 -1905-05-05 12:34:56.1512,1905-12-25 -1905-05-05 12:34:56.1513,1905-12-25 -1905-05-05 12:34:56.1514,1905-12-25 -1905-05-05 12:34:56.1515,1905-12-25 -1905-05-05 12:34:56.1516,1905-12-25 -1905-05-05 12:34:56.1517,1905-12-25 -1905-05-05 12:34:56.1518,1905-12-25 -1905-05-05 12:34:56.1519,1905-12-25 -1905-05-05 12:34:56.152,1905-12-25 -1905-05-05 12:34:56.1521,1905-12-25 -1905-05-05 12:34:56.1522,1905-12-25 -1905-05-05 12:34:56.1523,1905-12-25 -1905-05-05 12:34:56.1524,1905-12-25 -1905-05-05 12:34:56.1525,1905-12-25 -1905-05-05 12:34:56.1526,1905-12-25 -1905-05-05 12:34:56.1527,1905-12-25 -1905-05-05 12:34:56.1528,1905-12-25 -1905-05-05 12:34:56.1529,1905-12-25 -1905-05-05 12:34:56.153,1905-12-25 -1905-05-05 12:34:56.1531,1905-12-25 -1905-05-05 12:34:56.1532,1905-12-25 -1905-05-05 12:34:56.1533,1905-12-25 -1905-05-05 12:34:56.1534,1905-12-25 -1905-05-05 12:34:56.1535,1905-12-25 -1905-05-05 12:34:56.1536,1905-12-25 -1905-05-05 12:34:56.1537,1905-12-25 -1905-05-05 12:34:56.1538,1905-12-25 -1905-05-05 12:34:56.1539,1905-12-25 -1905-05-05 12:34:56.154,1905-12-25 -1905-05-05 12:34:56.1541,1905-12-25 -1905-05-05 12:34:56.1542,1905-12-25 -1905-05-05 12:34:56.1543,1905-12-25 -1905-05-05 12:34:56.1544,1905-12-25 -1905-05-05 12:34:56.1545,1905-12-25 -1905-05-05 12:34:56.1546,1905-12-25 -1905-05-05 12:34:56.1547,1905-12-25 -1905-05-05 12:34:56.1548,1905-12-25 -1905-05-05 12:34:56.1549,1905-12-25 -1905-05-05 12:34:56.155,1905-12-25 -1905-05-05 12:34:56.1551,1905-12-25 -1905-05-05 12:34:56.1552,1905-12-25 -1905-05-05 12:34:56.1553,1905-12-25 -1905-05-05 12:34:56.1554,1905-12-25 -1905-05-05 12:34:56.1555,1905-12-25 -1905-05-05 12:34:56.1556,1905-12-25 -1905-05-05 12:34:56.1557,1905-12-25 -1905-05-05 12:34:56.1558,1905-12-25 -1905-05-05 12:34:56.1559,1905-12-25 -1905-05-05 12:34:56.156,1905-12-25 -1905-05-05 12:34:56.1561,1905-12-25 -1905-05-05 12:34:56.1562,1905-12-25 -1905-05-05 12:34:56.1563,1905-12-25 -1905-05-05 12:34:56.1564,1905-12-25 -1905-05-05 12:34:56.1565,1905-12-25 -1905-05-05 12:34:56.1566,1905-12-25 -1905-05-05 12:34:56.1567,1905-12-25 -1905-05-05 12:34:56.1568,1905-12-25 -1905-05-05 12:34:56.1569,1905-12-25 -1905-05-05 12:34:56.157,1905-12-25 -1905-05-05 12:34:56.1571,1905-12-25 -1905-05-05 12:34:56.1572,1905-12-25 -1905-05-05 12:34:56.1573,1905-12-25 -1905-05-05 12:34:56.1574,1905-12-25 -1905-05-05 12:34:56.1575,1905-12-25 -1905-05-05 12:34:56.1576,1905-12-25 -1905-05-05 12:34:56.1577,1905-12-25 -1905-05-05 12:34:56.1578,1905-12-25 -1905-05-05 12:34:56.1579,1905-12-25 -1905-05-05 12:34:56.158,1905-12-25 -1905-05-05 12:34:56.1581,1905-12-25 -1905-05-05 12:34:56.1582,1905-12-25 -1905-05-05 12:34:56.1583,1905-12-25 -1905-05-05 12:34:56.1584,1905-12-25 -1905-05-05 12:34:56.1585,1905-12-25 -1905-05-05 12:34:56.1586,1905-12-25 -1905-05-05 12:34:56.1587,1905-12-25 -1905-05-05 12:34:56.1588,1905-12-25 -1905-05-05 12:34:56.1589,1905-12-25 -1905-05-05 12:34:56.159,1905-12-25 -1905-05-05 12:34:56.1591,1905-12-25 -1905-05-05 12:34:56.1592,1905-12-25 -1905-05-05 12:34:56.1593,1905-12-25 -1905-05-05 12:34:56.1594,1905-12-25 -1905-05-05 12:34:56.1595,1905-12-25 -1905-05-05 12:34:56.1596,1905-12-25 -1905-05-05 12:34:56.1597,1905-12-25 -1905-05-05 12:34:56.1598,1905-12-25 -1905-05-05 12:34:56.1599,1905-12-25 -1905-05-05 12:34:56.16,1905-12-25 -1905-05-05 12:34:56.1601,1905-12-25 -1905-05-05 12:34:56.1602,1905-12-25 -1905-05-05 12:34:56.1603,1905-12-25 -1905-05-05 12:34:56.1604,1905-12-25 -1905-05-05 12:34:56.1605,1905-12-25 -1905-05-05 12:34:56.1606,1905-12-25 -1905-05-05 12:34:56.1607,1905-12-25 -1905-05-05 12:34:56.1608,1905-12-25 -1905-05-05 12:34:56.1609,1905-12-25 -1905-05-05 12:34:56.161,1905-12-25 -1905-05-05 12:34:56.1611,1905-12-25 -1905-05-05 12:34:56.1612,1905-12-25 -1905-05-05 12:34:56.1613,1905-12-25 -1905-05-05 12:34:56.1614,1905-12-25 -1905-05-05 12:34:56.1615,1905-12-25 -1905-05-05 12:34:56.1616,1905-12-25 -1905-05-05 12:34:56.1617,1905-12-25 -1905-05-05 12:34:56.1618,1905-12-25 -1905-05-05 12:34:56.1619,1905-12-25 -1905-05-05 12:34:56.162,1905-12-25 -1905-05-05 12:34:56.1621,1905-12-25 -1905-05-05 12:34:56.1622,1905-12-25 -1905-05-05 12:34:56.1623,1905-12-25 -1905-05-05 12:34:56.1624,1905-12-25 -1905-05-05 12:34:56.1625,1905-12-25 -1905-05-05 12:34:56.1626,1905-12-25 -1905-05-05 12:34:56.1627,1905-12-25 -1905-05-05 12:34:56.1628,1905-12-25 -1905-05-05 12:34:56.1629,1905-12-25 -1905-05-05 12:34:56.163,1905-12-25 -1905-05-05 12:34:56.1631,1905-12-25 -1905-05-05 12:34:56.1632,1905-12-25 -1905-05-05 12:34:56.1633,1905-12-25 -1905-05-05 12:34:56.1634,1905-12-25 -1905-05-05 12:34:56.1635,1905-12-25 -1905-05-05 12:34:56.1636,1905-12-25 -1905-05-05 12:34:56.1637,1905-12-25 -1905-05-05 12:34:56.1638,1905-12-25 -1905-05-05 12:34:56.1639,1905-12-25 -1905-05-05 12:34:56.164,1905-12-25 -1905-05-05 12:34:56.1641,1905-12-25 -1905-05-05 12:34:56.1642,1905-12-25 -1905-05-05 12:34:56.1643,1905-12-25 -1905-05-05 12:34:56.1644,1905-12-25 -1905-05-05 12:34:56.1645,1905-12-25 -1905-05-05 12:34:56.1646,1905-12-25 -1905-05-05 12:34:56.1647,1905-12-25 -1905-05-05 12:34:56.1648,1905-12-25 -1905-05-05 12:34:56.1649,1905-12-25 -1905-05-05 12:34:56.165,1905-12-25 -1905-05-05 12:34:56.1651,1905-12-25 -1905-05-05 12:34:56.1652,1905-12-25 -1905-05-05 12:34:56.1653,1905-12-25 -1905-05-05 12:34:56.1654,1905-12-25 -1905-05-05 12:34:56.1655,1905-12-25 -1905-05-05 12:34:56.1656,1905-12-25 -1905-05-05 12:34:56.1657,1905-12-25 -1905-05-05 12:34:56.1658,1905-12-25 -1905-05-05 12:34:56.1659,1905-12-25 -1905-05-05 12:34:56.166,1905-12-25 -1905-05-05 12:34:56.1661,1905-12-25 -1905-05-05 12:34:56.1662,1905-12-25 -1905-05-05 12:34:56.1663,1905-12-25 -1905-05-05 12:34:56.1664,1905-12-25 -1905-05-05 12:34:56.1665,1905-12-25 -1905-05-05 12:34:56.1666,1905-12-25 -1905-05-05 12:34:56.1667,1905-12-25 -1905-05-05 12:34:56.1668,1905-12-25 -1905-05-05 12:34:56.1669,1905-12-25 -1905-05-05 12:34:56.167,1905-12-25 -1905-05-05 12:34:56.1671,1905-12-25 -1905-05-05 12:34:56.1672,1905-12-25 -1905-05-05 12:34:56.1673,1905-12-25 -1905-05-05 12:34:56.1674,1905-12-25 -1905-05-05 12:34:56.1675,1905-12-25 -1905-05-05 12:34:56.1676,1905-12-25 -1905-05-05 12:34:56.1677,1905-12-25 -1905-05-05 12:34:56.1678,1905-12-25 -1905-05-05 12:34:56.1679,1905-12-25 -1905-05-05 12:34:56.168,1905-12-25 -1905-05-05 12:34:56.1681,1905-12-25 -1905-05-05 12:34:56.1682,1905-12-25 -1905-05-05 12:34:56.1683,1905-12-25 -1905-05-05 12:34:56.1684,1905-12-25 -1905-05-05 12:34:56.1685,1905-12-25 -1905-05-05 12:34:56.1686,1905-12-25 -1905-05-05 12:34:56.1687,1905-12-25 -1905-05-05 12:34:56.1688,1905-12-25 -1905-05-05 12:34:56.1689,1905-12-25 -1905-05-05 12:34:56.169,1905-12-25 -1905-05-05 12:34:56.1691,1905-12-25 -1905-05-05 12:34:56.1692,1905-12-25 -1905-05-05 12:34:56.1693,1905-12-25 -1905-05-05 12:34:56.1694,1905-12-25 -1905-05-05 12:34:56.1695,1905-12-25 -1905-05-05 12:34:56.1696,1905-12-25 -1905-05-05 12:34:56.1697,1905-12-25 -1905-05-05 12:34:56.1698,1905-12-25 -1905-05-05 12:34:56.1699,1905-12-25 -1905-05-05 12:34:56.17,1905-12-25 -1905-05-05 12:34:56.1701,1905-12-25 -1905-05-05 12:34:56.1702,1905-12-25 -1905-05-05 12:34:56.1703,1905-12-25 -1905-05-05 12:34:56.1704,1905-12-25 -1905-05-05 12:34:56.1705,1905-12-25 -1905-05-05 12:34:56.1706,1905-12-25 -1905-05-05 12:34:56.1707,1905-12-25 -1905-05-05 12:34:56.1708,1905-12-25 -1905-05-05 12:34:56.1709,1905-12-25 -1905-05-05 12:34:56.171,1905-12-25 -1905-05-05 12:34:56.1711,1905-12-25 -1905-05-05 12:34:56.1712,1905-12-25 -1905-05-05 12:34:56.1713,1905-12-25 -1905-05-05 12:34:56.1714,1905-12-25 -1905-05-05 12:34:56.1715,1905-12-25 -1905-05-05 12:34:56.1716,1905-12-25 -1905-05-05 12:34:56.1717,1905-12-25 -1905-05-05 12:34:56.1718,1905-12-25 -1905-05-05 12:34:56.1719,1905-12-25 -1905-05-05 12:34:56.172,1905-12-25 -1905-05-05 12:34:56.1721,1905-12-25 -1905-05-05 12:34:56.1722,1905-12-25 -1905-05-05 12:34:56.1723,1905-12-25 -1905-05-05 12:34:56.1724,1905-12-25 -1905-05-05 12:34:56.1725,1905-12-25 -1905-05-05 12:34:56.1726,1905-12-25 -1905-05-05 12:34:56.1727,1905-12-25 -1905-05-05 12:34:56.1728,1905-12-25 -1905-05-05 12:34:56.1729,1905-12-25 -1905-05-05 12:34:56.173,1905-12-25 -1905-05-05 12:34:56.1731,1905-12-25 -1905-05-05 12:34:56.1732,1905-12-25 -1905-05-05 12:34:56.1733,1905-12-25 -1905-05-05 12:34:56.1734,1905-12-25 -1905-05-05 12:34:56.1735,1905-12-25 -1905-05-05 12:34:56.1736,1905-12-25 -1905-05-05 12:34:56.1737,1905-12-25 -1905-05-05 12:34:56.1738,1905-12-25 -1905-05-05 12:34:56.1739,1905-12-25 -1905-05-05 12:34:56.174,1905-12-25 -1905-05-05 12:34:56.1741,1905-12-25 -1905-05-05 12:34:56.1742,1905-12-25 -1905-05-05 12:34:56.1743,1905-12-25 -1905-05-05 12:34:56.1744,1905-12-25 -1905-05-05 12:34:56.1745,1905-12-25 -1905-05-05 12:34:56.1746,1905-12-25 -1905-05-05 12:34:56.1747,1905-12-25 -1905-05-05 12:34:56.1748,1905-12-25 -1905-05-05 12:34:56.1749,1905-12-25 -1905-05-05 12:34:56.175,1905-12-25 -1905-05-05 12:34:56.1751,1905-12-25 -1905-05-05 12:34:56.1752,1905-12-25 -1905-05-05 12:34:56.1753,1905-12-25 -1905-05-05 12:34:56.1754,1905-12-25 -1905-05-05 12:34:56.1755,1905-12-25 -1905-05-05 12:34:56.1756,1905-12-25 -1905-05-05 12:34:56.1757,1905-12-25 -1905-05-05 12:34:56.1758,1905-12-25 -1905-05-05 12:34:56.1759,1905-12-25 -1905-05-05 12:34:56.176,1905-12-25 -1905-05-05 12:34:56.1761,1905-12-25 -1905-05-05 12:34:56.1762,1905-12-25 -1905-05-05 12:34:56.1763,1905-12-25 -1905-05-05 12:34:56.1764,1905-12-25 -1905-05-05 12:34:56.1765,1905-12-25 -1905-05-05 12:34:56.1766,1905-12-25 -1905-05-05 12:34:56.1767,1905-12-25 -1905-05-05 12:34:56.1768,1905-12-25 -1905-05-05 12:34:56.1769,1905-12-25 -1905-05-05 12:34:56.177,1905-12-25 -1905-05-05 12:34:56.1771,1905-12-25 -1905-05-05 12:34:56.1772,1905-12-25 -1905-05-05 12:34:56.1773,1905-12-25 -1905-05-05 12:34:56.1774,1905-12-25 -1905-05-05 12:34:56.1775,1905-12-25 -1905-05-05 12:34:56.1776,1905-12-25 -1905-05-05 12:34:56.1777,1905-12-25 -1905-05-05 12:34:56.1778,1905-12-25 -1905-05-05 12:34:56.1779,1905-12-25 -1905-05-05 12:34:56.178,1905-12-25 -1905-05-05 12:34:56.1781,1905-12-25 -1905-05-05 12:34:56.1782,1905-12-25 -1905-05-05 12:34:56.1783,1905-12-25 -1905-05-05 12:34:56.1784,1905-12-25 -1905-05-05 12:34:56.1785,1905-12-25 -1905-05-05 12:34:56.1786,1905-12-25 -1905-05-05 12:34:56.1787,1905-12-25 -1905-05-05 12:34:56.1788,1905-12-25 -1905-05-05 12:34:56.1789,1905-12-25 -1905-05-05 12:34:56.179,1905-12-25 -1905-05-05 12:34:56.1791,1905-12-25 -1905-05-05 12:34:56.1792,1905-12-25 -1905-05-05 12:34:56.1793,1905-12-25 -1905-05-05 12:34:56.1794,1905-12-25 -1905-05-05 12:34:56.1795,1905-12-25 -1905-05-05 12:34:56.1796,1905-12-25 -1905-05-05 12:34:56.1797,1905-12-25 -1905-05-05 12:34:56.1798,1905-12-25 -1905-05-05 12:34:56.1799,1905-12-25 -1905-05-05 12:34:56.18,1905-12-25 -1905-05-05 12:34:56.1801,1905-12-25 -1905-05-05 12:34:56.1802,1905-12-25 -1905-05-05 12:34:56.1803,1905-12-25 -1905-05-05 12:34:56.1804,1905-12-25 -1905-05-05 12:34:56.1805,1905-12-25 -1905-05-05 12:34:56.1806,1905-12-25 -1905-05-05 12:34:56.1807,1905-12-25 -1905-05-05 12:34:56.1808,1905-12-25 -1905-05-05 12:34:56.1809,1905-12-25 -1905-05-05 12:34:56.181,1905-12-25 -1905-05-05 12:34:56.1811,1905-12-25 -1905-05-05 12:34:56.1812,1905-12-25 -1905-05-05 12:34:56.1813,1905-12-25 -1905-05-05 12:34:56.1814,1905-12-25 -1905-05-05 12:34:56.1815,1905-12-25 -1905-05-05 12:34:56.1816,1905-12-25 -1905-05-05 12:34:56.1817,1905-12-25 -1905-05-05 12:34:56.1818,1905-12-25 -1905-05-05 12:34:56.1819,1905-12-25 -1905-05-05 12:34:56.182,1905-12-25 -1905-05-05 12:34:56.1821,1905-12-25 -1905-05-05 12:34:56.1822,1905-12-25 -1905-05-05 12:34:56.1823,1905-12-25 -1905-05-05 12:34:56.1824,1905-12-25 -1905-05-05 12:34:56.1825,1905-12-25 -1905-05-05 12:34:56.1826,1905-12-25 -1905-05-05 12:34:56.1827,1905-12-25 -1905-05-05 12:34:56.1828,1905-12-25 -1905-05-05 12:34:56.1829,1905-12-25 -1905-05-05 12:34:56.183,1905-12-25 -1905-05-05 12:34:56.1831,1905-12-25 -1905-05-05 12:34:56.1832,1905-12-25 -1905-05-05 12:34:56.1833,1905-12-25 -1905-05-05 12:34:56.1834,1905-12-25 -1905-05-05 12:34:56.1835,1905-12-25 -1905-05-05 12:34:56.1836,1905-12-25 -1905-05-05 12:34:56.1837,1905-12-25 -1905-05-05 12:34:56.1838,1905-12-25 -1905-05-05 12:34:56.1839,1905-12-25 -1905-05-05 12:34:56.184,1905-12-25 -1905-05-05 12:34:56.1841,1905-12-25 -1905-05-05 12:34:56.1842,1905-12-25 -1905-05-05 12:34:56.1843,1905-12-25 -1905-05-05 12:34:56.1844,1905-12-25 -1905-05-05 12:34:56.1845,1905-12-25 -1905-05-05 12:34:56.1846,1905-12-25 -1905-05-05 12:34:56.1847,1905-12-25 -1905-05-05 12:34:56.1848,1905-12-25 -1905-05-05 12:34:56.1849,1905-12-25 -1905-05-05 12:34:56.185,1905-12-25 -1905-05-05 12:34:56.1851,1905-12-25 -1905-05-05 12:34:56.1852,1905-12-25 -1905-05-05 12:34:56.1853,1905-12-25 -1905-05-05 12:34:56.1854,1905-12-25 -1905-05-05 12:34:56.1855,1905-12-25 -1905-05-05 12:34:56.1856,1905-12-25 -1905-05-05 12:34:56.1857,1905-12-25 -1905-05-05 12:34:56.1858,1905-12-25 -1905-05-05 12:34:56.1859,1905-12-25 -1905-05-05 12:34:56.186,1905-12-25 -1905-05-05 12:34:56.1861,1905-12-25 -1905-05-05 12:34:56.1862,1905-12-25 -1905-05-05 12:34:56.1863,1905-12-25 -1905-05-05 12:34:56.1864,1905-12-25 -1905-05-05 12:34:56.1865,1905-12-25 -1905-05-05 12:34:56.1866,1905-12-25 -1905-05-05 12:34:56.1867,1905-12-25 -1905-05-05 12:34:56.1868,1905-12-25 -1905-05-05 12:34:56.1869,1905-12-25 -1905-05-05 12:34:56.187,1905-12-25 -1905-05-05 12:34:56.1871,1905-12-25 -1905-05-05 12:34:56.1872,1905-12-25 -1905-05-05 12:34:56.1873,1905-12-25 -1905-05-05 12:34:56.1874,1905-12-25 -1905-05-05 12:34:56.1875,1905-12-25 -1905-05-05 12:34:56.1876,1905-12-25 -1905-05-05 12:34:56.1877,1905-12-25 -1905-05-05 12:34:56.1878,1905-12-25 -1905-05-05 12:34:56.1879,1905-12-25 -1905-05-05 12:34:56.188,1905-12-25 -1905-05-05 12:34:56.1881,1905-12-25 -1905-05-05 12:34:56.1882,1905-12-25 -1905-05-05 12:34:56.1883,1905-12-25 -1905-05-05 12:34:56.1884,1905-12-25 -1905-05-05 12:34:56.1885,1905-12-25 -1905-05-05 12:34:56.1886,1905-12-25 -1905-05-05 12:34:56.1887,1905-12-25 -1905-05-05 12:34:56.1888,1905-12-25 -1905-05-05 12:34:56.1889,1905-12-25 -1905-05-05 12:34:56.189,1905-12-25 -1905-05-05 12:34:56.1891,1905-12-25 -1905-05-05 12:34:56.1892,1905-12-25 -1905-05-05 12:34:56.1893,1905-12-25 -1905-05-05 12:34:56.1894,1905-12-25 -1905-05-05 12:34:56.1895,1905-12-25 -1905-05-05 12:34:56.1896,1905-12-25 -1905-05-05 12:34:56.1897,1905-12-25 -1905-05-05 12:34:56.1898,1905-12-25 -1905-05-05 12:34:56.1899,1905-12-25 -1905-05-05 12:34:56.19,1905-12-25 -1905-05-05 12:34:56.1901,1905-12-25 -1905-05-05 12:34:56.1902,1905-12-25 -1905-05-05 12:34:56.1903,1905-12-25 -1905-05-05 12:34:56.1904,1905-12-25 -1905-05-05 12:34:56.1905,1905-12-25 -1905-05-05 12:34:56.1906,1905-12-25 -1905-05-05 12:34:56.1907,1905-12-25 -1905-05-05 12:34:56.1908,1905-12-25 -1905-05-05 12:34:56.1909,1905-12-25 -1905-05-05 12:34:56.191,1905-12-25 -1905-05-05 12:34:56.1911,1905-12-25 -1905-05-05 12:34:56.1912,1905-12-25 -1905-05-05 12:34:56.1913,1905-12-25 -1905-05-05 12:34:56.1914,1905-12-25 -1905-05-05 12:34:56.1915,1905-12-25 -1905-05-05 12:34:56.1916,1905-12-25 -1905-05-05 12:34:56.1917,1905-12-25 -1905-05-05 12:34:56.1918,1905-12-25 -1905-05-05 12:34:56.1919,1905-12-25 -1905-05-05 12:34:56.192,1905-12-25 -1905-05-05 12:34:56.1921,1905-12-25 -1905-05-05 12:34:56.1922,1905-12-25 -1905-05-05 12:34:56.1923,1905-12-25 -1905-05-05 12:34:56.1924,1905-12-25 -1905-05-05 12:34:56.1925,1905-12-25 -1905-05-05 12:34:56.1926,1905-12-25 -1905-05-05 12:34:56.1927,1905-12-25 -1905-05-05 12:34:56.1928,1905-12-25 -1905-05-05 12:34:56.1929,1905-12-25 -1905-05-05 12:34:56.193,1905-12-25 -1905-05-05 12:34:56.1931,1905-12-25 -1905-05-05 12:34:56.1932,1905-12-25 -1905-05-05 12:34:56.1933,1905-12-25 -1905-05-05 12:34:56.1934,1905-12-25 -1905-05-05 12:34:56.1935,1905-12-25 -1905-05-05 12:34:56.1936,1905-12-25 -1905-05-05 12:34:56.1937,1905-12-25 -1905-05-05 12:34:56.1938,1905-12-25 -1905-05-05 12:34:56.1939,1905-12-25 -1905-05-05 12:34:56.194,1905-12-25 -1905-05-05 12:34:56.1941,1905-12-25 -1905-05-05 12:34:56.1942,1905-12-25 -1905-05-05 12:34:56.1943,1905-12-25 -1905-05-05 12:34:56.1944,1905-12-25 -1905-05-05 12:34:56.1945,1905-12-25 -1905-05-05 12:34:56.1946,1905-12-25 -1905-05-05 12:34:56.1947,1905-12-25 -1905-05-05 12:34:56.1948,1905-12-25 -1905-05-05 12:34:56.1949,1905-12-25 -1905-05-05 12:34:56.195,1905-12-25 -1905-05-05 12:34:56.1951,1905-12-25 -1905-05-05 12:34:56.1952,1905-12-25 -1905-05-05 12:34:56.1953,1905-12-25 -1905-05-05 12:34:56.1954,1905-12-25 -1905-05-05 12:34:56.1955,1905-12-25 -1905-05-05 12:34:56.1956,1905-12-25 -1905-05-05 12:34:56.1957,1905-12-25 -1905-05-05 12:34:56.1958,1905-12-25 -1905-05-05 12:34:56.1959,1905-12-25 -1905-05-05 12:34:56.196,1905-12-25 -1905-05-05 12:34:56.1961,1905-12-25 -1905-05-05 12:34:56.1962,1905-12-25 -1905-05-05 12:34:56.1963,1905-12-25 -1905-05-05 12:34:56.1964,1905-12-25 -1905-05-05 12:34:56.1965,1905-12-25 -1905-05-05 12:34:56.1966,1905-12-25 -1905-05-05 12:34:56.1967,1905-12-25 -1905-05-05 12:34:56.1968,1905-12-25 -1905-05-05 12:34:56.1969,1905-12-25 -1905-05-05 12:34:56.197,1905-12-25 -1905-05-05 12:34:56.1971,1905-12-25 -1905-05-05 12:34:56.1972,1905-12-25 -1905-05-05 12:34:56.1973,1905-12-25 -1905-05-05 12:34:56.1974,1905-12-25 -1905-05-05 12:34:56.1975,1905-12-25 -1905-05-05 12:34:56.1976,1905-12-25 -1905-05-05 12:34:56.1977,1905-12-25 -1905-05-05 12:34:56.1978,1905-12-25 -1905-05-05 12:34:56.1979,1905-12-25 -1905-05-05 12:34:56.198,1905-12-25 -1905-05-05 12:34:56.1981,1905-12-25 -1905-05-05 12:34:56.1982,1905-12-25 -1905-05-05 12:34:56.1983,1905-12-25 -1905-05-05 12:34:56.1984,1905-12-25 -1905-05-05 12:34:56.1985,1905-12-25 -1905-05-05 12:34:56.1986,1905-12-25 -1905-05-05 12:34:56.1987,1905-12-25 -1905-05-05 12:34:56.1988,1905-12-25 -1905-05-05 12:34:56.1989,1905-12-25 -1905-05-05 12:34:56.199,1905-12-25 -1905-05-05 12:34:56.1991,1905-12-25 -1905-05-05 12:34:56.1992,1905-12-25 -1905-05-05 12:34:56.1993,1905-12-25 -1905-05-05 12:34:56.1994,1905-12-25 -1905-05-05 12:34:56.1995,1905-12-25 -1905-05-05 12:34:56.1996,1905-12-25 -1905-05-05 12:34:56.1997,1905-12-25 -1905-05-05 12:34:56.1998,1905-12-25 -1905-05-05 12:34:56.1999,1905-12-25 -1906-05-05 12:34:56.1,1906-12-25 -1906-05-05 12:34:56.1001,1906-12-25 -1906-05-05 12:34:56.1002,1906-12-25 -1906-05-05 12:34:56.1003,1906-12-25 -1906-05-05 12:34:56.1004,1906-12-25 -1906-05-05 12:34:56.1005,1906-12-25 -1906-05-05 12:34:56.1006,1906-12-25 -1906-05-05 12:34:56.1007,1906-12-25 -1906-05-05 12:34:56.1008,1906-12-25 -1906-05-05 12:34:56.1009,1906-12-25 -1906-05-05 12:34:56.101,1906-12-25 -1906-05-05 12:34:56.1011,1906-12-25 -1906-05-05 12:34:56.1012,1906-12-25 -1906-05-05 12:34:56.1013,1906-12-25 -1906-05-05 12:34:56.1014,1906-12-25 -1906-05-05 12:34:56.1015,1906-12-25 -1906-05-05 12:34:56.1016,1906-12-25 -1906-05-05 12:34:56.1017,1906-12-25 -1906-05-05 12:34:56.1018,1906-12-25 -1906-05-05 12:34:56.1019,1906-12-25 -1906-05-05 12:34:56.102,1906-12-25 -1906-05-05 12:34:56.1021,1906-12-25 -1906-05-05 12:34:56.1022,1906-12-25 -1906-05-05 12:34:56.1023,1906-12-25 -1906-05-05 12:34:56.1024,1906-12-25 -1906-05-05 12:34:56.1025,1906-12-25 -1906-05-05 12:34:56.1026,1906-12-25 -1906-05-05 12:34:56.1027,1906-12-25 -1906-05-05 12:34:56.1028,1906-12-25 -1906-05-05 12:34:56.1029,1906-12-25 -1906-05-05 12:34:56.103,1906-12-25 -1906-05-05 12:34:56.1031,1906-12-25 -1906-05-05 12:34:56.1032,1906-12-25 -1906-05-05 12:34:56.1033,1906-12-25 -1906-05-05 12:34:56.1034,1906-12-25 -1906-05-05 12:34:56.1035,1906-12-25 -1906-05-05 12:34:56.1036,1906-12-25 -1906-05-05 12:34:56.1037,1906-12-25 -1906-05-05 12:34:56.1038,1906-12-25 -1906-05-05 12:34:56.1039,1906-12-25 -1906-05-05 12:34:56.104,1906-12-25 -1906-05-05 12:34:56.1041,1906-12-25 -1906-05-05 12:34:56.1042,1906-12-25 -1906-05-05 12:34:56.1043,1906-12-25 -1906-05-05 12:34:56.1044,1906-12-25 -1906-05-05 12:34:56.1045,1906-12-25 -1906-05-05 12:34:56.1046,1906-12-25 -1906-05-05 12:34:56.1047,1906-12-25 -1906-05-05 12:34:56.1048,1906-12-25 -1906-05-05 12:34:56.1049,1906-12-25 -1906-05-05 12:34:56.105,1906-12-25 -1906-05-05 12:34:56.1051,1906-12-25 -1906-05-05 12:34:56.1052,1906-12-25 -1906-05-05 12:34:56.1053,1906-12-25 -1906-05-05 12:34:56.1054,1906-12-25 -1906-05-05 12:34:56.1055,1906-12-25 -1906-05-05 12:34:56.1056,1906-12-25 -1906-05-05 12:34:56.1057,1906-12-25 -1906-05-05 12:34:56.1058,1906-12-25 -1906-05-05 12:34:56.1059,1906-12-25 -1906-05-05 12:34:56.106,1906-12-25 -1906-05-05 12:34:56.1061,1906-12-25 -1906-05-05 12:34:56.1062,1906-12-25 -1906-05-05 12:34:56.1063,1906-12-25 -1906-05-05 12:34:56.1064,1906-12-25 -1906-05-05 12:34:56.1065,1906-12-25 -1906-05-05 12:34:56.1066,1906-12-25 -1906-05-05 12:34:56.1067,1906-12-25 -1906-05-05 12:34:56.1068,1906-12-25 -1906-05-05 12:34:56.1069,1906-12-25 -1906-05-05 12:34:56.107,1906-12-25 -1906-05-05 12:34:56.1071,1906-12-25 -1906-05-05 12:34:56.1072,1906-12-25 -1906-05-05 12:34:56.1073,1906-12-25 -1906-05-05 12:34:56.1074,1906-12-25 -1906-05-05 12:34:56.1075,1906-12-25 -1906-05-05 12:34:56.1076,1906-12-25 -1906-05-05 12:34:56.1077,1906-12-25 -1906-05-05 12:34:56.1078,1906-12-25 -1906-05-05 12:34:56.1079,1906-12-25 -1906-05-05 12:34:56.108,1906-12-25 -1906-05-05 12:34:56.1081,1906-12-25 -1906-05-05 12:34:56.1082,1906-12-25 -1906-05-05 12:34:56.1083,1906-12-25 -1906-05-05 12:34:56.1084,1906-12-25 -1906-05-05 12:34:56.1085,1906-12-25 -1906-05-05 12:34:56.1086,1906-12-25 -1906-05-05 12:34:56.1087,1906-12-25 -1906-05-05 12:34:56.1088,1906-12-25 -1906-05-05 12:34:56.1089,1906-12-25 -1906-05-05 12:34:56.109,1906-12-25 -1906-05-05 12:34:56.1091,1906-12-25 -1906-05-05 12:34:56.1092,1906-12-25 -1906-05-05 12:34:56.1093,1906-12-25 -1906-05-05 12:34:56.1094,1906-12-25 -1906-05-05 12:34:56.1095,1906-12-25 -1906-05-05 12:34:56.1096,1906-12-25 -1906-05-05 12:34:56.1097,1906-12-25 -1906-05-05 12:34:56.1098,1906-12-25 -1906-05-05 12:34:56.1099,1906-12-25 -1906-05-05 12:34:56.11,1906-12-25 -1906-05-05 12:34:56.1101,1906-12-25 -1906-05-05 12:34:56.1102,1906-12-25 -1906-05-05 12:34:56.1103,1906-12-25 -1906-05-05 12:34:56.1104,1906-12-25 -1906-05-05 12:34:56.1105,1906-12-25 -1906-05-05 12:34:56.1106,1906-12-25 -1906-05-05 12:34:56.1107,1906-12-25 -1906-05-05 12:34:56.1108,1906-12-25 -1906-05-05 12:34:56.1109,1906-12-25 -1906-05-05 12:34:56.111,1906-12-25 -1906-05-05 12:34:56.1111,1906-12-25 -1906-05-05 12:34:56.1112,1906-12-25 -1906-05-05 12:34:56.1113,1906-12-25 -1906-05-05 12:34:56.1114,1906-12-25 -1906-05-05 12:34:56.1115,1906-12-25 -1906-05-05 12:34:56.1116,1906-12-25 -1906-05-05 12:34:56.1117,1906-12-25 -1906-05-05 12:34:56.1118,1906-12-25 -1906-05-05 12:34:56.1119,1906-12-25 -1906-05-05 12:34:56.112,1906-12-25 -1906-05-05 12:34:56.1121,1906-12-25 -1906-05-05 12:34:56.1122,1906-12-25 -1906-05-05 12:34:56.1123,1906-12-25 -1906-05-05 12:34:56.1124,1906-12-25 -1906-05-05 12:34:56.1125,1906-12-25 -1906-05-05 12:34:56.1126,1906-12-25 -1906-05-05 12:34:56.1127,1906-12-25 -1906-05-05 12:34:56.1128,1906-12-25 -1906-05-05 12:34:56.1129,1906-12-25 -1906-05-05 12:34:56.113,1906-12-25 -1906-05-05 12:34:56.1131,1906-12-25 -1906-05-05 12:34:56.1132,1906-12-25 -1906-05-05 12:34:56.1133,1906-12-25 -1906-05-05 12:34:56.1134,1906-12-25 -1906-05-05 12:34:56.1135,1906-12-25 -1906-05-05 12:34:56.1136,1906-12-25 -1906-05-05 12:34:56.1137,1906-12-25 -1906-05-05 12:34:56.1138,1906-12-25 -1906-05-05 12:34:56.1139,1906-12-25 -1906-05-05 12:34:56.114,1906-12-25 -1906-05-05 12:34:56.1141,1906-12-25 -1906-05-05 12:34:56.1142,1906-12-25 -1906-05-05 12:34:56.1143,1906-12-25 -1906-05-05 12:34:56.1144,1906-12-25 -1906-05-05 12:34:56.1145,1906-12-25 -1906-05-05 12:34:56.1146,1906-12-25 -1906-05-05 12:34:56.1147,1906-12-25 -1906-05-05 12:34:56.1148,1906-12-25 -1906-05-05 12:34:56.1149,1906-12-25 -1906-05-05 12:34:56.115,1906-12-25 -1906-05-05 12:34:56.1151,1906-12-25 -1906-05-05 12:34:56.1152,1906-12-25 -1906-05-05 12:34:56.1153,1906-12-25 -1906-05-05 12:34:56.1154,1906-12-25 -1906-05-05 12:34:56.1155,1906-12-25 -1906-05-05 12:34:56.1156,1906-12-25 -1906-05-05 12:34:56.1157,1906-12-25 -1906-05-05 12:34:56.1158,1906-12-25 -1906-05-05 12:34:56.1159,1906-12-25 -1906-05-05 12:34:56.116,1906-12-25 -1906-05-05 12:34:56.1161,1906-12-25 -1906-05-05 12:34:56.1162,1906-12-25 -1906-05-05 12:34:56.1163,1906-12-25 -1906-05-05 12:34:56.1164,1906-12-25 -1906-05-05 12:34:56.1165,1906-12-25 -1906-05-05 12:34:56.1166,1906-12-25 -1906-05-05 12:34:56.1167,1906-12-25 -1906-05-05 12:34:56.1168,1906-12-25 -1906-05-05 12:34:56.1169,1906-12-25 -1906-05-05 12:34:56.117,1906-12-25 -1906-05-05 12:34:56.1171,1906-12-25 -1906-05-05 12:34:56.1172,1906-12-25 -1906-05-05 12:34:56.1173,1906-12-25 -1906-05-05 12:34:56.1174,1906-12-25 -1906-05-05 12:34:56.1175,1906-12-25 -1906-05-05 12:34:56.1176,1906-12-25 -1906-05-05 12:34:56.1177,1906-12-25 -1906-05-05 12:34:56.1178,1906-12-25 -1906-05-05 12:34:56.1179,1906-12-25 -1906-05-05 12:34:56.118,1906-12-25 -1906-05-05 12:34:56.1181,1906-12-25 -1906-05-05 12:34:56.1182,1906-12-25 -1906-05-05 12:34:56.1183,1906-12-25 -1906-05-05 12:34:56.1184,1906-12-25 -1906-05-05 12:34:56.1185,1906-12-25 -1906-05-05 12:34:56.1186,1906-12-25 -1906-05-05 12:34:56.1187,1906-12-25 -1906-05-05 12:34:56.1188,1906-12-25 -1906-05-05 12:34:56.1189,1906-12-25 -1906-05-05 12:34:56.119,1906-12-25 -1906-05-05 12:34:56.1191,1906-12-25 -1906-05-05 12:34:56.1192,1906-12-25 -1906-05-05 12:34:56.1193,1906-12-25 -1906-05-05 12:34:56.1194,1906-12-25 -1906-05-05 12:34:56.1195,1906-12-25 -1906-05-05 12:34:56.1196,1906-12-25 -1906-05-05 12:34:56.1197,1906-12-25 -1906-05-05 12:34:56.1198,1906-12-25 -1906-05-05 12:34:56.1199,1906-12-25 -1906-05-05 12:34:56.12,1906-12-25 -1906-05-05 12:34:56.1201,1906-12-25 -1906-05-05 12:34:56.1202,1906-12-25 -1906-05-05 12:34:56.1203,1906-12-25 -1906-05-05 12:34:56.1204,1906-12-25 -1906-05-05 12:34:56.1205,1906-12-25 -1906-05-05 12:34:56.1206,1906-12-25 -1906-05-05 12:34:56.1207,1906-12-25 -1906-05-05 12:34:56.1208,1906-12-25 -1906-05-05 12:34:56.1209,1906-12-25 -1906-05-05 12:34:56.121,1906-12-25 -1906-05-05 12:34:56.1211,1906-12-25 -1906-05-05 12:34:56.1212,1906-12-25 -1906-05-05 12:34:56.1213,1906-12-25 -1906-05-05 12:34:56.1214,1906-12-25 -1906-05-05 12:34:56.1215,1906-12-25 -1906-05-05 12:34:56.1216,1906-12-25 -1906-05-05 12:34:56.1217,1906-12-25 -1906-05-05 12:34:56.1218,1906-12-25 -1906-05-05 12:34:56.1219,1906-12-25 -1906-05-05 12:34:56.122,1906-12-25 -1906-05-05 12:34:56.1221,1906-12-25 -1906-05-05 12:34:56.1222,1906-12-25 -1906-05-05 12:34:56.1223,1906-12-25 -1906-05-05 12:34:56.1224,1906-12-25 -1906-05-05 12:34:56.1225,1906-12-25 -1906-05-05 12:34:56.1226,1906-12-25 -1906-05-05 12:34:56.1227,1906-12-25 -1906-05-05 12:34:56.1228,1906-12-25 -1906-05-05 12:34:56.1229,1906-12-25 -1906-05-05 12:34:56.123,1906-12-25 -1906-05-05 12:34:56.1231,1906-12-25 -1906-05-05 12:34:56.1232,1906-12-25 -1906-05-05 12:34:56.1233,1906-12-25 -1906-05-05 12:34:56.1234,1906-12-25 -1906-05-05 12:34:56.1235,1906-12-25 -1906-05-05 12:34:56.1236,1906-12-25 -1906-05-05 12:34:56.1237,1906-12-25 -1906-05-05 12:34:56.1238,1906-12-25 -1906-05-05 12:34:56.1239,1906-12-25 -1906-05-05 12:34:56.124,1906-12-25 -1906-05-05 12:34:56.1241,1906-12-25 -1906-05-05 12:34:56.1242,1906-12-25 -1906-05-05 12:34:56.1243,1906-12-25 -1906-05-05 12:34:56.1244,1906-12-25 -1906-05-05 12:34:56.1245,1906-12-25 -1906-05-05 12:34:56.1246,1906-12-25 -1906-05-05 12:34:56.1247,1906-12-25 -1906-05-05 12:34:56.1248,1906-12-25 -1906-05-05 12:34:56.1249,1906-12-25 -1906-05-05 12:34:56.125,1906-12-25 -1906-05-05 12:34:56.1251,1906-12-25 -1906-05-05 12:34:56.1252,1906-12-25 -1906-05-05 12:34:56.1253,1906-12-25 -1906-05-05 12:34:56.1254,1906-12-25 -1906-05-05 12:34:56.1255,1906-12-25 -1906-05-05 12:34:56.1256,1906-12-25 -1906-05-05 12:34:56.1257,1906-12-25 -1906-05-05 12:34:56.1258,1906-12-25 -1906-05-05 12:34:56.1259,1906-12-25 -1906-05-05 12:34:56.126,1906-12-25 -1906-05-05 12:34:56.1261,1906-12-25 -1906-05-05 12:34:56.1262,1906-12-25 -1906-05-05 12:34:56.1263,1906-12-25 -1906-05-05 12:34:56.1264,1906-12-25 -1906-05-05 12:34:56.1265,1906-12-25 -1906-05-05 12:34:56.1266,1906-12-25 -1906-05-05 12:34:56.1267,1906-12-25 -1906-05-05 12:34:56.1268,1906-12-25 -1906-05-05 12:34:56.1269,1906-12-25 -1906-05-05 12:34:56.127,1906-12-25 -1906-05-05 12:34:56.1271,1906-12-25 -1906-05-05 12:34:56.1272,1906-12-25 -1906-05-05 12:34:56.1273,1906-12-25 -1906-05-05 12:34:56.1274,1906-12-25 -1906-05-05 12:34:56.1275,1906-12-25 -1906-05-05 12:34:56.1276,1906-12-25 -1906-05-05 12:34:56.1277,1906-12-25 -1906-05-05 12:34:56.1278,1906-12-25 -1906-05-05 12:34:56.1279,1906-12-25 -1906-05-05 12:34:56.128,1906-12-25 -1906-05-05 12:34:56.1281,1906-12-25 -1906-05-05 12:34:56.1282,1906-12-25 -1906-05-05 12:34:56.1283,1906-12-25 -1906-05-05 12:34:56.1284,1906-12-25 -1906-05-05 12:34:56.1285,1906-12-25 -1906-05-05 12:34:56.1286,1906-12-25 -1906-05-05 12:34:56.1287,1906-12-25 -1906-05-05 12:34:56.1288,1906-12-25 -1906-05-05 12:34:56.1289,1906-12-25 -1906-05-05 12:34:56.129,1906-12-25 -1906-05-05 12:34:56.1291,1906-12-25 -1906-05-05 12:34:56.1292,1906-12-25 -1906-05-05 12:34:56.1293,1906-12-25 -1906-05-05 12:34:56.1294,1906-12-25 -1906-05-05 12:34:56.1295,1906-12-25 -1906-05-05 12:34:56.1296,1906-12-25 -1906-05-05 12:34:56.1297,1906-12-25 -1906-05-05 12:34:56.1298,1906-12-25 -1906-05-05 12:34:56.1299,1906-12-25 -1906-05-05 12:34:56.13,1906-12-25 -1906-05-05 12:34:56.1301,1906-12-25 -1906-05-05 12:34:56.1302,1906-12-25 -1906-05-05 12:34:56.1303,1906-12-25 -1906-05-05 12:34:56.1304,1906-12-25 -1906-05-05 12:34:56.1305,1906-12-25 -1906-05-05 12:34:56.1306,1906-12-25 -1906-05-05 12:34:56.1307,1906-12-25 -1906-05-05 12:34:56.1308,1906-12-25 -1906-05-05 12:34:56.1309,1906-12-25 -1906-05-05 12:34:56.131,1906-12-25 -1906-05-05 12:34:56.1311,1906-12-25 -1906-05-05 12:34:56.1312,1906-12-25 -1906-05-05 12:34:56.1313,1906-12-25 -1906-05-05 12:34:56.1314,1906-12-25 -1906-05-05 12:34:56.1315,1906-12-25 -1906-05-05 12:34:56.1316,1906-12-25 -1906-05-05 12:34:56.1317,1906-12-25 -1906-05-05 12:34:56.1318,1906-12-25 -1906-05-05 12:34:56.1319,1906-12-25 -1906-05-05 12:34:56.132,1906-12-25 -1906-05-05 12:34:56.1321,1906-12-25 -1906-05-05 12:34:56.1322,1906-12-25 -1906-05-05 12:34:56.1323,1906-12-25 -1906-05-05 12:34:56.1324,1906-12-25 -1906-05-05 12:34:56.1325,1906-12-25 -1906-05-05 12:34:56.1326,1906-12-25 -1906-05-05 12:34:56.1327,1906-12-25 -1906-05-05 12:34:56.1328,1906-12-25 -1906-05-05 12:34:56.1329,1906-12-25 -1906-05-05 12:34:56.133,1906-12-25 -1906-05-05 12:34:56.1331,1906-12-25 -1906-05-05 12:34:56.1332,1906-12-25 -1906-05-05 12:34:56.1333,1906-12-25 -1906-05-05 12:34:56.1334,1906-12-25 -1906-05-05 12:34:56.1335,1906-12-25 -1906-05-05 12:34:56.1336,1906-12-25 -1906-05-05 12:34:56.1337,1906-12-25 -1906-05-05 12:34:56.1338,1906-12-25 -1906-05-05 12:34:56.1339,1906-12-25 -1906-05-05 12:34:56.134,1906-12-25 -1906-05-05 12:34:56.1341,1906-12-25 -1906-05-05 12:34:56.1342,1906-12-25 -1906-05-05 12:34:56.1343,1906-12-25 -1906-05-05 12:34:56.1344,1906-12-25 -1906-05-05 12:34:56.1345,1906-12-25 -1906-05-05 12:34:56.1346,1906-12-25 -1906-05-05 12:34:56.1347,1906-12-25 -1906-05-05 12:34:56.1348,1906-12-25 -1906-05-05 12:34:56.1349,1906-12-25 -1906-05-05 12:34:56.135,1906-12-25 -1906-05-05 12:34:56.1351,1906-12-25 -1906-05-05 12:34:56.1352,1906-12-25 -1906-05-05 12:34:56.1353,1906-12-25 -1906-05-05 12:34:56.1354,1906-12-25 -1906-05-05 12:34:56.1355,1906-12-25 -1906-05-05 12:34:56.1356,1906-12-25 -1906-05-05 12:34:56.1357,1906-12-25 -1906-05-05 12:34:56.1358,1906-12-25 -1906-05-05 12:34:56.1359,1906-12-25 -1906-05-05 12:34:56.136,1906-12-25 -1906-05-05 12:34:56.1361,1906-12-25 -1906-05-05 12:34:56.1362,1906-12-25 -1906-05-05 12:34:56.1363,1906-12-25 -1906-05-05 12:34:56.1364,1906-12-25 -1906-05-05 12:34:56.1365,1906-12-25 -1906-05-05 12:34:56.1366,1906-12-25 -1906-05-05 12:34:56.1367,1906-12-25 -1906-05-05 12:34:56.1368,1906-12-25 -1906-05-05 12:34:56.1369,1906-12-25 -1906-05-05 12:34:56.137,1906-12-25 -1906-05-05 12:34:56.1371,1906-12-25 -1906-05-05 12:34:56.1372,1906-12-25 -1906-05-05 12:34:56.1373,1906-12-25 -1906-05-05 12:34:56.1374,1906-12-25 -1906-05-05 12:34:56.1375,1906-12-25 -1906-05-05 12:34:56.1376,1906-12-25 -1906-05-05 12:34:56.1377,1906-12-25 -1906-05-05 12:34:56.1378,1906-12-25 -1906-05-05 12:34:56.1379,1906-12-25 -1906-05-05 12:34:56.138,1906-12-25 -1906-05-05 12:34:56.1381,1906-12-25 -1906-05-05 12:34:56.1382,1906-12-25 -1906-05-05 12:34:56.1383,1906-12-25 -1906-05-05 12:34:56.1384,1906-12-25 -1906-05-05 12:34:56.1385,1906-12-25 -1906-05-05 12:34:56.1386,1906-12-25 -1906-05-05 12:34:56.1387,1906-12-25 -1906-05-05 12:34:56.1388,1906-12-25 -1906-05-05 12:34:56.1389,1906-12-25 -1906-05-05 12:34:56.139,1906-12-25 -1906-05-05 12:34:56.1391,1906-12-25 -1906-05-05 12:34:56.1392,1906-12-25 -1906-05-05 12:34:56.1393,1906-12-25 -1906-05-05 12:34:56.1394,1906-12-25 -1906-05-05 12:34:56.1395,1906-12-25 -1906-05-05 12:34:56.1396,1906-12-25 -1906-05-05 12:34:56.1397,1906-12-25 -1906-05-05 12:34:56.1398,1906-12-25 -1906-05-05 12:34:56.1399,1906-12-25 -1906-05-05 12:34:56.14,1906-12-25 -1906-05-05 12:34:56.1401,1906-12-25 -1906-05-05 12:34:56.1402,1906-12-25 -1906-05-05 12:34:56.1403,1906-12-25 -1906-05-05 12:34:56.1404,1906-12-25 -1906-05-05 12:34:56.1405,1906-12-25 -1906-05-05 12:34:56.1406,1906-12-25 -1906-05-05 12:34:56.1407,1906-12-25 -1906-05-05 12:34:56.1408,1906-12-25 -1906-05-05 12:34:56.1409,1906-12-25 -1906-05-05 12:34:56.141,1906-12-25 -1906-05-05 12:34:56.1411,1906-12-25 -1906-05-05 12:34:56.1412,1906-12-25 -1906-05-05 12:34:56.1413,1906-12-25 -1906-05-05 12:34:56.1414,1906-12-25 -1906-05-05 12:34:56.1415,1906-12-25 -1906-05-05 12:34:56.1416,1906-12-25 -1906-05-05 12:34:56.1417,1906-12-25 -1906-05-05 12:34:56.1418,1906-12-25 -1906-05-05 12:34:56.1419,1906-12-25 -1906-05-05 12:34:56.142,1906-12-25 -1906-05-05 12:34:56.1421,1906-12-25 -1906-05-05 12:34:56.1422,1906-12-25 -1906-05-05 12:34:56.1423,1906-12-25 -1906-05-05 12:34:56.1424,1906-12-25 -1906-05-05 12:34:56.1425,1906-12-25 -1906-05-05 12:34:56.1426,1906-12-25 -1906-05-05 12:34:56.1427,1906-12-25 -1906-05-05 12:34:56.1428,1906-12-25 -1906-05-05 12:34:56.1429,1906-12-25 -1906-05-05 12:34:56.143,1906-12-25 -1906-05-05 12:34:56.1431,1906-12-25 -1906-05-05 12:34:56.1432,1906-12-25 -1906-05-05 12:34:56.1433,1906-12-25 -1906-05-05 12:34:56.1434,1906-12-25 -1906-05-05 12:34:56.1435,1906-12-25 -1906-05-05 12:34:56.1436,1906-12-25 -1906-05-05 12:34:56.1437,1906-12-25 -1906-05-05 12:34:56.1438,1906-12-25 -1906-05-05 12:34:56.1439,1906-12-25 -1906-05-05 12:34:56.144,1906-12-25 -1906-05-05 12:34:56.1441,1906-12-25 -1906-05-05 12:34:56.1442,1906-12-25 -1906-05-05 12:34:56.1443,1906-12-25 -1906-05-05 12:34:56.1444,1906-12-25 -1906-05-05 12:34:56.1445,1906-12-25 -1906-05-05 12:34:56.1446,1906-12-25 -1906-05-05 12:34:56.1447,1906-12-25 -1906-05-05 12:34:56.1448,1906-12-25 -1906-05-05 12:34:56.1449,1906-12-25 -1906-05-05 12:34:56.145,1906-12-25 -1906-05-05 12:34:56.1451,1906-12-25 -1906-05-05 12:34:56.1452,1906-12-25 -1906-05-05 12:34:56.1453,1906-12-25 -1906-05-05 12:34:56.1454,1906-12-25 -1906-05-05 12:34:56.1455,1906-12-25 -1906-05-05 12:34:56.1456,1906-12-25 -1906-05-05 12:34:56.1457,1906-12-25 -1906-05-05 12:34:56.1458,1906-12-25 -1906-05-05 12:34:56.1459,1906-12-25 -1906-05-05 12:34:56.146,1906-12-25 -1906-05-05 12:34:56.1461,1906-12-25 -1906-05-05 12:34:56.1462,1906-12-25 -1906-05-05 12:34:56.1463,1906-12-25 -1906-05-05 12:34:56.1464,1906-12-25 -1906-05-05 12:34:56.1465,1906-12-25 -1906-05-05 12:34:56.1466,1906-12-25 -1906-05-05 12:34:56.1467,1906-12-25 -1906-05-05 12:34:56.1468,1906-12-25 -1906-05-05 12:34:56.1469,1906-12-25 -1906-05-05 12:34:56.147,1906-12-25 -1906-05-05 12:34:56.1471,1906-12-25 -1906-05-05 12:34:56.1472,1906-12-25 -1906-05-05 12:34:56.1473,1906-12-25 -1906-05-05 12:34:56.1474,1906-12-25 -1906-05-05 12:34:56.1475,1906-12-25 -1906-05-05 12:34:56.1476,1906-12-25 -1906-05-05 12:34:56.1477,1906-12-25 -1906-05-05 12:34:56.1478,1906-12-25 -1906-05-05 12:34:56.1479,1906-12-25 -1906-05-05 12:34:56.148,1906-12-25 -1906-05-05 12:34:56.1481,1906-12-25 -1906-05-05 12:34:56.1482,1906-12-25 -1906-05-05 12:34:56.1483,1906-12-25 -1906-05-05 12:34:56.1484,1906-12-25 -1906-05-05 12:34:56.1485,1906-12-25 -1906-05-05 12:34:56.1486,1906-12-25 -1906-05-05 12:34:56.1487,1906-12-25 -1906-05-05 12:34:56.1488,1906-12-25 -1906-05-05 12:34:56.1489,1906-12-25 -1906-05-05 12:34:56.149,1906-12-25 -1906-05-05 12:34:56.1491,1906-12-25 -1906-05-05 12:34:56.1492,1906-12-25 -1906-05-05 12:34:56.1493,1906-12-25 -1906-05-05 12:34:56.1494,1906-12-25 -1906-05-05 12:34:56.1495,1906-12-25 -1906-05-05 12:34:56.1496,1906-12-25 -1906-05-05 12:34:56.1497,1906-12-25 -1906-05-05 12:34:56.1498,1906-12-25 -1906-05-05 12:34:56.1499,1906-12-25 -1906-05-05 12:34:56.15,1906-12-25 -1906-05-05 12:34:56.1501,1906-12-25 -1906-05-05 12:34:56.1502,1906-12-25 -1906-05-05 12:34:56.1503,1906-12-25 -1906-05-05 12:34:56.1504,1906-12-25 -1906-05-05 12:34:56.1505,1906-12-25 -1906-05-05 12:34:56.1506,1906-12-25 -1906-05-05 12:34:56.1507,1906-12-25 -1906-05-05 12:34:56.1508,1906-12-25 -1906-05-05 12:34:56.1509,1906-12-25 -1906-05-05 12:34:56.151,1906-12-25 -1906-05-05 12:34:56.1511,1906-12-25 -1906-05-05 12:34:56.1512,1906-12-25 -1906-05-05 12:34:56.1513,1906-12-25 -1906-05-05 12:34:56.1514,1906-12-25 -1906-05-05 12:34:56.1515,1906-12-25 -1906-05-05 12:34:56.1516,1906-12-25 -1906-05-05 12:34:56.1517,1906-12-25 -1906-05-05 12:34:56.1518,1906-12-25 -1906-05-05 12:34:56.1519,1906-12-25 -1906-05-05 12:34:56.152,1906-12-25 -1906-05-05 12:34:56.1521,1906-12-25 -1906-05-05 12:34:56.1522,1906-12-25 -1906-05-05 12:34:56.1523,1906-12-25 -1906-05-05 12:34:56.1524,1906-12-25 -1906-05-05 12:34:56.1525,1906-12-25 -1906-05-05 12:34:56.1526,1906-12-25 -1906-05-05 12:34:56.1527,1906-12-25 -1906-05-05 12:34:56.1528,1906-12-25 -1906-05-05 12:34:56.1529,1906-12-25 -1906-05-05 12:34:56.153,1906-12-25 -1906-05-05 12:34:56.1531,1906-12-25 -1906-05-05 12:34:56.1532,1906-12-25 -1906-05-05 12:34:56.1533,1906-12-25 -1906-05-05 12:34:56.1534,1906-12-25 -1906-05-05 12:34:56.1535,1906-12-25 -1906-05-05 12:34:56.1536,1906-12-25 -1906-05-05 12:34:56.1537,1906-12-25 -1906-05-05 12:34:56.1538,1906-12-25 -1906-05-05 12:34:56.1539,1906-12-25 -1906-05-05 12:34:56.154,1906-12-25 -1906-05-05 12:34:56.1541,1906-12-25 -1906-05-05 12:34:56.1542,1906-12-25 -1906-05-05 12:34:56.1543,1906-12-25 -1906-05-05 12:34:56.1544,1906-12-25 -1906-05-05 12:34:56.1545,1906-12-25 -1906-05-05 12:34:56.1546,1906-12-25 -1906-05-05 12:34:56.1547,1906-12-25 -1906-05-05 12:34:56.1548,1906-12-25 -1906-05-05 12:34:56.1549,1906-12-25 -1906-05-05 12:34:56.155,1906-12-25 -1906-05-05 12:34:56.1551,1906-12-25 -1906-05-05 12:34:56.1552,1906-12-25 -1906-05-05 12:34:56.1553,1906-12-25 -1906-05-05 12:34:56.1554,1906-12-25 -1906-05-05 12:34:56.1555,1906-12-25 -1906-05-05 12:34:56.1556,1906-12-25 -1906-05-05 12:34:56.1557,1906-12-25 -1906-05-05 12:34:56.1558,1906-12-25 -1906-05-05 12:34:56.1559,1906-12-25 -1906-05-05 12:34:56.156,1906-12-25 -1906-05-05 12:34:56.1561,1906-12-25 -1906-05-05 12:34:56.1562,1906-12-25 -1906-05-05 12:34:56.1563,1906-12-25 -1906-05-05 12:34:56.1564,1906-12-25 -1906-05-05 12:34:56.1565,1906-12-25 -1906-05-05 12:34:56.1566,1906-12-25 -1906-05-05 12:34:56.1567,1906-12-25 -1906-05-05 12:34:56.1568,1906-12-25 -1906-05-05 12:34:56.1569,1906-12-25 -1906-05-05 12:34:56.157,1906-12-25 -1906-05-05 12:34:56.1571,1906-12-25 -1906-05-05 12:34:56.1572,1906-12-25 -1906-05-05 12:34:56.1573,1906-12-25 -1906-05-05 12:34:56.1574,1906-12-25 -1906-05-05 12:34:56.1575,1906-12-25 -1906-05-05 12:34:56.1576,1906-12-25 -1906-05-05 12:34:56.1577,1906-12-25 -1906-05-05 12:34:56.1578,1906-12-25 -1906-05-05 12:34:56.1579,1906-12-25 -1906-05-05 12:34:56.158,1906-12-25 -1906-05-05 12:34:56.1581,1906-12-25 -1906-05-05 12:34:56.1582,1906-12-25 -1906-05-05 12:34:56.1583,1906-12-25 -1906-05-05 12:34:56.1584,1906-12-25 -1906-05-05 12:34:56.1585,1906-12-25 -1906-05-05 12:34:56.1586,1906-12-25 -1906-05-05 12:34:56.1587,1906-12-25 -1906-05-05 12:34:56.1588,1906-12-25 -1906-05-05 12:34:56.1589,1906-12-25 -1906-05-05 12:34:56.159,1906-12-25 -1906-05-05 12:34:56.1591,1906-12-25 -1906-05-05 12:34:56.1592,1906-12-25 -1906-05-05 12:34:56.1593,1906-12-25 -1906-05-05 12:34:56.1594,1906-12-25 -1906-05-05 12:34:56.1595,1906-12-25 -1906-05-05 12:34:56.1596,1906-12-25 -1906-05-05 12:34:56.1597,1906-12-25 -1906-05-05 12:34:56.1598,1906-12-25 -1906-05-05 12:34:56.1599,1906-12-25 -1906-05-05 12:34:56.16,1906-12-25 -1906-05-05 12:34:56.1601,1906-12-25 -1906-05-05 12:34:56.1602,1906-12-25 -1906-05-05 12:34:56.1603,1906-12-25 -1906-05-05 12:34:56.1604,1906-12-25 -1906-05-05 12:34:56.1605,1906-12-25 -1906-05-05 12:34:56.1606,1906-12-25 -1906-05-05 12:34:56.1607,1906-12-25 -1906-05-05 12:34:56.1608,1906-12-25 -1906-05-05 12:34:56.1609,1906-12-25 -1906-05-05 12:34:56.161,1906-12-25 -1906-05-05 12:34:56.1611,1906-12-25 -1906-05-05 12:34:56.1612,1906-12-25 -1906-05-05 12:34:56.1613,1906-12-25 -1906-05-05 12:34:56.1614,1906-12-25 -1906-05-05 12:34:56.1615,1906-12-25 -1906-05-05 12:34:56.1616,1906-12-25 -1906-05-05 12:34:56.1617,1906-12-25 -1906-05-05 12:34:56.1618,1906-12-25 -1906-05-05 12:34:56.1619,1906-12-25 -1906-05-05 12:34:56.162,1906-12-25 -1906-05-05 12:34:56.1621,1906-12-25 -1906-05-05 12:34:56.1622,1906-12-25 -1906-05-05 12:34:56.1623,1906-12-25 -1906-05-05 12:34:56.1624,1906-12-25 -1906-05-05 12:34:56.1625,1906-12-25 -1906-05-05 12:34:56.1626,1906-12-25 -1906-05-05 12:34:56.1627,1906-12-25 -1906-05-05 12:34:56.1628,1906-12-25 -1906-05-05 12:34:56.1629,1906-12-25 -1906-05-05 12:34:56.163,1906-12-25 -1906-05-05 12:34:56.1631,1906-12-25 -1906-05-05 12:34:56.1632,1906-12-25 -1906-05-05 12:34:56.1633,1906-12-25 -1906-05-05 12:34:56.1634,1906-12-25 -1906-05-05 12:34:56.1635,1906-12-25 -1906-05-05 12:34:56.1636,1906-12-25 -1906-05-05 12:34:56.1637,1906-12-25 -1906-05-05 12:34:56.1638,1906-12-25 -1906-05-05 12:34:56.1639,1906-12-25 -1906-05-05 12:34:56.164,1906-12-25 -1906-05-05 12:34:56.1641,1906-12-25 -1906-05-05 12:34:56.1642,1906-12-25 -1906-05-05 12:34:56.1643,1906-12-25 -1906-05-05 12:34:56.1644,1906-12-25 -1906-05-05 12:34:56.1645,1906-12-25 -1906-05-05 12:34:56.1646,1906-12-25 -1906-05-05 12:34:56.1647,1906-12-25 -1906-05-05 12:34:56.1648,1906-12-25 -1906-05-05 12:34:56.1649,1906-12-25 -1906-05-05 12:34:56.165,1906-12-25 -1906-05-05 12:34:56.1651,1906-12-25 -1906-05-05 12:34:56.1652,1906-12-25 -1906-05-05 12:34:56.1653,1906-12-25 -1906-05-05 12:34:56.1654,1906-12-25 -1906-05-05 12:34:56.1655,1906-12-25 -1906-05-05 12:34:56.1656,1906-12-25 -1906-05-05 12:34:56.1657,1906-12-25 -1906-05-05 12:34:56.1658,1906-12-25 -1906-05-05 12:34:56.1659,1906-12-25 -1906-05-05 12:34:56.166,1906-12-25 -1906-05-05 12:34:56.1661,1906-12-25 -1906-05-05 12:34:56.1662,1906-12-25 -1906-05-05 12:34:56.1663,1906-12-25 -1906-05-05 12:34:56.1664,1906-12-25 -1906-05-05 12:34:56.1665,1906-12-25 -1906-05-05 12:34:56.1666,1906-12-25 -1906-05-05 12:34:56.1667,1906-12-25 -1906-05-05 12:34:56.1668,1906-12-25 -1906-05-05 12:34:56.1669,1906-12-25 -1906-05-05 12:34:56.167,1906-12-25 -1906-05-05 12:34:56.1671,1906-12-25 -1906-05-05 12:34:56.1672,1906-12-25 -1906-05-05 12:34:56.1673,1906-12-25 -1906-05-05 12:34:56.1674,1906-12-25 -1906-05-05 12:34:56.1675,1906-12-25 -1906-05-05 12:34:56.1676,1906-12-25 -1906-05-05 12:34:56.1677,1906-12-25 -1906-05-05 12:34:56.1678,1906-12-25 -1906-05-05 12:34:56.1679,1906-12-25 -1906-05-05 12:34:56.168,1906-12-25 -1906-05-05 12:34:56.1681,1906-12-25 -1906-05-05 12:34:56.1682,1906-12-25 -1906-05-05 12:34:56.1683,1906-12-25 -1906-05-05 12:34:56.1684,1906-12-25 -1906-05-05 12:34:56.1685,1906-12-25 -1906-05-05 12:34:56.1686,1906-12-25 -1906-05-05 12:34:56.1687,1906-12-25 -1906-05-05 12:34:56.1688,1906-12-25 -1906-05-05 12:34:56.1689,1906-12-25 -1906-05-05 12:34:56.169,1906-12-25 -1906-05-05 12:34:56.1691,1906-12-25 -1906-05-05 12:34:56.1692,1906-12-25 -1906-05-05 12:34:56.1693,1906-12-25 -1906-05-05 12:34:56.1694,1906-12-25 -1906-05-05 12:34:56.1695,1906-12-25 -1906-05-05 12:34:56.1696,1906-12-25 -1906-05-05 12:34:56.1697,1906-12-25 -1906-05-05 12:34:56.1698,1906-12-25 -1906-05-05 12:34:56.1699,1906-12-25 -1906-05-05 12:34:56.17,1906-12-25 -1906-05-05 12:34:56.1701,1906-12-25 -1906-05-05 12:34:56.1702,1906-12-25 -1906-05-05 12:34:56.1703,1906-12-25 -1906-05-05 12:34:56.1704,1906-12-25 -1906-05-05 12:34:56.1705,1906-12-25 -1906-05-05 12:34:56.1706,1906-12-25 -1906-05-05 12:34:56.1707,1906-12-25 -1906-05-05 12:34:56.1708,1906-12-25 -1906-05-05 12:34:56.1709,1906-12-25 -1906-05-05 12:34:56.171,1906-12-25 -1906-05-05 12:34:56.1711,1906-12-25 -1906-05-05 12:34:56.1712,1906-12-25 -1906-05-05 12:34:56.1713,1906-12-25 -1906-05-05 12:34:56.1714,1906-12-25 -1906-05-05 12:34:56.1715,1906-12-25 -1906-05-05 12:34:56.1716,1906-12-25 -1906-05-05 12:34:56.1717,1906-12-25 -1906-05-05 12:34:56.1718,1906-12-25 -1906-05-05 12:34:56.1719,1906-12-25 -1906-05-05 12:34:56.172,1906-12-25 -1906-05-05 12:34:56.1721,1906-12-25 -1906-05-05 12:34:56.1722,1906-12-25 -1906-05-05 12:34:56.1723,1906-12-25 -1906-05-05 12:34:56.1724,1906-12-25 -1906-05-05 12:34:56.1725,1906-12-25 -1906-05-05 12:34:56.1726,1906-12-25 -1906-05-05 12:34:56.1727,1906-12-25 -1906-05-05 12:34:56.1728,1906-12-25 -1906-05-05 12:34:56.1729,1906-12-25 -1906-05-05 12:34:56.173,1906-12-25 -1906-05-05 12:34:56.1731,1906-12-25 -1906-05-05 12:34:56.1732,1906-12-25 -1906-05-05 12:34:56.1733,1906-12-25 -1906-05-05 12:34:56.1734,1906-12-25 -1906-05-05 12:34:56.1735,1906-12-25 -1906-05-05 12:34:56.1736,1906-12-25 -1906-05-05 12:34:56.1737,1906-12-25 -1906-05-05 12:34:56.1738,1906-12-25 -1906-05-05 12:34:56.1739,1906-12-25 -1906-05-05 12:34:56.174,1906-12-25 -1906-05-05 12:34:56.1741,1906-12-25 -1906-05-05 12:34:56.1742,1906-12-25 -1906-05-05 12:34:56.1743,1906-12-25 -1906-05-05 12:34:56.1744,1906-12-25 -1906-05-05 12:34:56.1745,1906-12-25 -1906-05-05 12:34:56.1746,1906-12-25 -1906-05-05 12:34:56.1747,1906-12-25 -1906-05-05 12:34:56.1748,1906-12-25 -1906-05-05 12:34:56.1749,1906-12-25 -1906-05-05 12:34:56.175,1906-12-25 -1906-05-05 12:34:56.1751,1906-12-25 -1906-05-05 12:34:56.1752,1906-12-25 -1906-05-05 12:34:56.1753,1906-12-25 -1906-05-05 12:34:56.1754,1906-12-25 -1906-05-05 12:34:56.1755,1906-12-25 -1906-05-05 12:34:56.1756,1906-12-25 -1906-05-05 12:34:56.1757,1906-12-25 -1906-05-05 12:34:56.1758,1906-12-25 -1906-05-05 12:34:56.1759,1906-12-25 -1906-05-05 12:34:56.176,1906-12-25 -1906-05-05 12:34:56.1761,1906-12-25 -1906-05-05 12:34:56.1762,1906-12-25 -1906-05-05 12:34:56.1763,1906-12-25 -1906-05-05 12:34:56.1764,1906-12-25 -1906-05-05 12:34:56.1765,1906-12-25 -1906-05-05 12:34:56.1766,1906-12-25 -1906-05-05 12:34:56.1767,1906-12-25 -1906-05-05 12:34:56.1768,1906-12-25 -1906-05-05 12:34:56.1769,1906-12-25 -1906-05-05 12:34:56.177,1906-12-25 -1906-05-05 12:34:56.1771,1906-12-25 -1906-05-05 12:34:56.1772,1906-12-25 -1906-05-05 12:34:56.1773,1906-12-25 -1906-05-05 12:34:56.1774,1906-12-25 -1906-05-05 12:34:56.1775,1906-12-25 -1906-05-05 12:34:56.1776,1906-12-25 -1906-05-05 12:34:56.1777,1906-12-25 -1906-05-05 12:34:56.1778,1906-12-25 -1906-05-05 12:34:56.1779,1906-12-25 -1906-05-05 12:34:56.178,1906-12-25 -1906-05-05 12:34:56.1781,1906-12-25 -1906-05-05 12:34:56.1782,1906-12-25 -1906-05-05 12:34:56.1783,1906-12-25 -1906-05-05 12:34:56.1784,1906-12-25 -1906-05-05 12:34:56.1785,1906-12-25 -1906-05-05 12:34:56.1786,1906-12-25 -1906-05-05 12:34:56.1787,1906-12-25 -1906-05-05 12:34:56.1788,1906-12-25 -1906-05-05 12:34:56.1789,1906-12-25 -1906-05-05 12:34:56.179,1906-12-25 -1906-05-05 12:34:56.1791,1906-12-25 -1906-05-05 12:34:56.1792,1906-12-25 -1906-05-05 12:34:56.1793,1906-12-25 -1906-05-05 12:34:56.1794,1906-12-25 -1906-05-05 12:34:56.1795,1906-12-25 -1906-05-05 12:34:56.1796,1906-12-25 -1906-05-05 12:34:56.1797,1906-12-25 -1906-05-05 12:34:56.1798,1906-12-25 -1906-05-05 12:34:56.1799,1906-12-25 -1906-05-05 12:34:56.18,1906-12-25 -1906-05-05 12:34:56.1801,1906-12-25 -1906-05-05 12:34:56.1802,1906-12-25 -1906-05-05 12:34:56.1803,1906-12-25 -1906-05-05 12:34:56.1804,1906-12-25 -1906-05-05 12:34:56.1805,1906-12-25 -1906-05-05 12:34:56.1806,1906-12-25 -1906-05-05 12:34:56.1807,1906-12-25 -1906-05-05 12:34:56.1808,1906-12-25 -1906-05-05 12:34:56.1809,1906-12-25 -1906-05-05 12:34:56.181,1906-12-25 -1906-05-05 12:34:56.1811,1906-12-25 -1906-05-05 12:34:56.1812,1906-12-25 -1906-05-05 12:34:56.1813,1906-12-25 -1906-05-05 12:34:56.1814,1906-12-25 -1906-05-05 12:34:56.1815,1906-12-25 -1906-05-05 12:34:56.1816,1906-12-25 -1906-05-05 12:34:56.1817,1906-12-25 -1906-05-05 12:34:56.1818,1906-12-25 -1906-05-05 12:34:56.1819,1906-12-25 -1906-05-05 12:34:56.182,1906-12-25 -1906-05-05 12:34:56.1821,1906-12-25 -1906-05-05 12:34:56.1822,1906-12-25 -1906-05-05 12:34:56.1823,1906-12-25 -1906-05-05 12:34:56.1824,1906-12-25 -1906-05-05 12:34:56.1825,1906-12-25 -1906-05-05 12:34:56.1826,1906-12-25 -1906-05-05 12:34:56.1827,1906-12-25 -1906-05-05 12:34:56.1828,1906-12-25 -1906-05-05 12:34:56.1829,1906-12-25 -1906-05-05 12:34:56.183,1906-12-25 -1906-05-05 12:34:56.1831,1906-12-25 -1906-05-05 12:34:56.1832,1906-12-25 -1906-05-05 12:34:56.1833,1906-12-25 -1906-05-05 12:34:56.1834,1906-12-25 -1906-05-05 12:34:56.1835,1906-12-25 -1906-05-05 12:34:56.1836,1906-12-25 -1906-05-05 12:34:56.1837,1906-12-25 -1906-05-05 12:34:56.1838,1906-12-25 -1906-05-05 12:34:56.1839,1906-12-25 -1906-05-05 12:34:56.184,1906-12-25 -1906-05-05 12:34:56.1841,1906-12-25 -1906-05-05 12:34:56.1842,1906-12-25 -1906-05-05 12:34:56.1843,1906-12-25 -1906-05-05 12:34:56.1844,1906-12-25 -1906-05-05 12:34:56.1845,1906-12-25 -1906-05-05 12:34:56.1846,1906-12-25 -1906-05-05 12:34:56.1847,1906-12-25 -1906-05-05 12:34:56.1848,1906-12-25 -1906-05-05 12:34:56.1849,1906-12-25 -1906-05-05 12:34:56.185,1906-12-25 -1906-05-05 12:34:56.1851,1906-12-25 -1906-05-05 12:34:56.1852,1906-12-25 -1906-05-05 12:34:56.1853,1906-12-25 -1906-05-05 12:34:56.1854,1906-12-25 -1906-05-05 12:34:56.1855,1906-12-25 -1906-05-05 12:34:56.1856,1906-12-25 -1906-05-05 12:34:56.1857,1906-12-25 -1906-05-05 12:34:56.1858,1906-12-25 -1906-05-05 12:34:56.1859,1906-12-25 -1906-05-05 12:34:56.186,1906-12-25 -1906-05-05 12:34:56.1861,1906-12-25 -1906-05-05 12:34:56.1862,1906-12-25 -1906-05-05 12:34:56.1863,1906-12-25 -1906-05-05 12:34:56.1864,1906-12-25 -1906-05-05 12:34:56.1865,1906-12-25 -1906-05-05 12:34:56.1866,1906-12-25 -1906-05-05 12:34:56.1867,1906-12-25 -1906-05-05 12:34:56.1868,1906-12-25 -1906-05-05 12:34:56.1869,1906-12-25 -1906-05-05 12:34:56.187,1906-12-25 -1906-05-05 12:34:56.1871,1906-12-25 -1906-05-05 12:34:56.1872,1906-12-25 -1906-05-05 12:34:56.1873,1906-12-25 -1906-05-05 12:34:56.1874,1906-12-25 -1906-05-05 12:34:56.1875,1906-12-25 -1906-05-05 12:34:56.1876,1906-12-25 -1906-05-05 12:34:56.1877,1906-12-25 -1906-05-05 12:34:56.1878,1906-12-25 -1906-05-05 12:34:56.1879,1906-12-25 -1906-05-05 12:34:56.188,1906-12-25 -1906-05-05 12:34:56.1881,1906-12-25 -1906-05-05 12:34:56.1882,1906-12-25 -1906-05-05 12:34:56.1883,1906-12-25 -1906-05-05 12:34:56.1884,1906-12-25 -1906-05-05 12:34:56.1885,1906-12-25 -1906-05-05 12:34:56.1886,1906-12-25 -1906-05-05 12:34:56.1887,1906-12-25 -1906-05-05 12:34:56.1888,1906-12-25 -1906-05-05 12:34:56.1889,1906-12-25 -1906-05-05 12:34:56.189,1906-12-25 -1906-05-05 12:34:56.1891,1906-12-25 -1906-05-05 12:34:56.1892,1906-12-25 -1906-05-05 12:34:56.1893,1906-12-25 -1906-05-05 12:34:56.1894,1906-12-25 -1906-05-05 12:34:56.1895,1906-12-25 -1906-05-05 12:34:56.1896,1906-12-25 -1906-05-05 12:34:56.1897,1906-12-25 -1906-05-05 12:34:56.1898,1906-12-25 -1906-05-05 12:34:56.1899,1906-12-25 -1906-05-05 12:34:56.19,1906-12-25 -1906-05-05 12:34:56.1901,1906-12-25 -1906-05-05 12:34:56.1902,1906-12-25 -1906-05-05 12:34:56.1903,1906-12-25 -1906-05-05 12:34:56.1904,1906-12-25 -1906-05-05 12:34:56.1905,1906-12-25 -1906-05-05 12:34:56.1906,1906-12-25 -1906-05-05 12:34:56.1907,1906-12-25 -1906-05-05 12:34:56.1908,1906-12-25 -1906-05-05 12:34:56.1909,1906-12-25 -1906-05-05 12:34:56.191,1906-12-25 -1906-05-05 12:34:56.1911,1906-12-25 -1906-05-05 12:34:56.1912,1906-12-25 -1906-05-05 12:34:56.1913,1906-12-25 -1906-05-05 12:34:56.1914,1906-12-25 -1906-05-05 12:34:56.1915,1906-12-25 -1906-05-05 12:34:56.1916,1906-12-25 -1906-05-05 12:34:56.1917,1906-12-25 -1906-05-05 12:34:56.1918,1906-12-25 -1906-05-05 12:34:56.1919,1906-12-25 -1906-05-05 12:34:56.192,1906-12-25 -1906-05-05 12:34:56.1921,1906-12-25 -1906-05-05 12:34:56.1922,1906-12-25 -1906-05-05 12:34:56.1923,1906-12-25 -1906-05-05 12:34:56.1924,1906-12-25 -1906-05-05 12:34:56.1925,1906-12-25 -1906-05-05 12:34:56.1926,1906-12-25 -1906-05-05 12:34:56.1927,1906-12-25 -1906-05-05 12:34:56.1928,1906-12-25 -1906-05-05 12:34:56.1929,1906-12-25 -1906-05-05 12:34:56.193,1906-12-25 -1906-05-05 12:34:56.1931,1906-12-25 -1906-05-05 12:34:56.1932,1906-12-25 -1906-05-05 12:34:56.1933,1906-12-25 -1906-05-05 12:34:56.1934,1906-12-25 -1906-05-05 12:34:56.1935,1906-12-25 -1906-05-05 12:34:56.1936,1906-12-25 -1906-05-05 12:34:56.1937,1906-12-25 -1906-05-05 12:34:56.1938,1906-12-25 -1906-05-05 12:34:56.1939,1906-12-25 -1906-05-05 12:34:56.194,1906-12-25 -1906-05-05 12:34:56.1941,1906-12-25 -1906-05-05 12:34:56.1942,1906-12-25 -1906-05-05 12:34:56.1943,1906-12-25 -1906-05-05 12:34:56.1944,1906-12-25 -1906-05-05 12:34:56.1945,1906-12-25 -1906-05-05 12:34:56.1946,1906-12-25 -1906-05-05 12:34:56.1947,1906-12-25 -1906-05-05 12:34:56.1948,1906-12-25 -1906-05-05 12:34:56.1949,1906-12-25 -1906-05-05 12:34:56.195,1906-12-25 -1906-05-05 12:34:56.1951,1906-12-25 -1906-05-05 12:34:56.1952,1906-12-25 -1906-05-05 12:34:56.1953,1906-12-25 -1906-05-05 12:34:56.1954,1906-12-25 -1906-05-05 12:34:56.1955,1906-12-25 -1906-05-05 12:34:56.1956,1906-12-25 -1906-05-05 12:34:56.1957,1906-12-25 -1906-05-05 12:34:56.1958,1906-12-25 -1906-05-05 12:34:56.1959,1906-12-25 -1906-05-05 12:34:56.196,1906-12-25 -1906-05-05 12:34:56.1961,1906-12-25 -1906-05-05 12:34:56.1962,1906-12-25 -1906-05-05 12:34:56.1963,1906-12-25 -1906-05-05 12:34:56.1964,1906-12-25 -1906-05-05 12:34:56.1965,1906-12-25 -1906-05-05 12:34:56.1966,1906-12-25 -1906-05-05 12:34:56.1967,1906-12-25 -1906-05-05 12:34:56.1968,1906-12-25 -1906-05-05 12:34:56.1969,1906-12-25 -1906-05-05 12:34:56.197,1906-12-25 -1906-05-05 12:34:56.1971,1906-12-25 -1906-05-05 12:34:56.1972,1906-12-25 -1906-05-05 12:34:56.1973,1906-12-25 -1906-05-05 12:34:56.1974,1906-12-25 -1906-05-05 12:34:56.1975,1906-12-25 -1906-05-05 12:34:56.1976,1906-12-25 -1906-05-05 12:34:56.1977,1906-12-25 -1906-05-05 12:34:56.1978,1906-12-25 -1906-05-05 12:34:56.1979,1906-12-25 -1906-05-05 12:34:56.198,1906-12-25 -1906-05-05 12:34:56.1981,1906-12-25 -1906-05-05 12:34:56.1982,1906-12-25 -1906-05-05 12:34:56.1983,1906-12-25 -1906-05-05 12:34:56.1984,1906-12-25 -1906-05-05 12:34:56.1985,1906-12-25 -1906-05-05 12:34:56.1986,1906-12-25 -1906-05-05 12:34:56.1987,1906-12-25 -1906-05-05 12:34:56.1988,1906-12-25 -1906-05-05 12:34:56.1989,1906-12-25 -1906-05-05 12:34:56.199,1906-12-25 -1906-05-05 12:34:56.1991,1906-12-25 -1906-05-05 12:34:56.1992,1906-12-25 -1906-05-05 12:34:56.1993,1906-12-25 -1906-05-05 12:34:56.1994,1906-12-25 -1906-05-05 12:34:56.1995,1906-12-25 -1906-05-05 12:34:56.1996,1906-12-25 -1906-05-05 12:34:56.1997,1906-12-25 -1906-05-05 12:34:56.1998,1906-12-25 -1906-05-05 12:34:56.1999,1906-12-25 -1907-05-05 12:34:56.1,1907-12-25 -1907-05-05 12:34:56.1001,1907-12-25 -1907-05-05 12:34:56.1002,1907-12-25 -1907-05-05 12:34:56.1003,1907-12-25 -1907-05-05 12:34:56.1004,1907-12-25 -1907-05-05 12:34:56.1005,1907-12-25 -1907-05-05 12:34:56.1006,1907-12-25 -1907-05-05 12:34:56.1007,1907-12-25 -1907-05-05 12:34:56.1008,1907-12-25 -1907-05-05 12:34:56.1009,1907-12-25 -1907-05-05 12:34:56.101,1907-12-25 -1907-05-05 12:34:56.1011,1907-12-25 -1907-05-05 12:34:56.1012,1907-12-25 -1907-05-05 12:34:56.1013,1907-12-25 -1907-05-05 12:34:56.1014,1907-12-25 -1907-05-05 12:34:56.1015,1907-12-25 -1907-05-05 12:34:56.1016,1907-12-25 -1907-05-05 12:34:56.1017,1907-12-25 -1907-05-05 12:34:56.1018,1907-12-25 -1907-05-05 12:34:56.1019,1907-12-25 -1907-05-05 12:34:56.102,1907-12-25 -1907-05-05 12:34:56.1021,1907-12-25 -1907-05-05 12:34:56.1022,1907-12-25 -1907-05-05 12:34:56.1023,1907-12-25 -1907-05-05 12:34:56.1024,1907-12-25 -1907-05-05 12:34:56.1025,1907-12-25 -1907-05-05 12:34:56.1026,1907-12-25 -1907-05-05 12:34:56.1027,1907-12-25 -1907-05-05 12:34:56.1028,1907-12-25 -1907-05-05 12:34:56.1029,1907-12-25 -1907-05-05 12:34:56.103,1907-12-25 -1907-05-05 12:34:56.1031,1907-12-25 -1907-05-05 12:34:56.1032,1907-12-25 -1907-05-05 12:34:56.1033,1907-12-25 -1907-05-05 12:34:56.1034,1907-12-25 -1907-05-05 12:34:56.1035,1907-12-25 -1907-05-05 12:34:56.1036,1907-12-25 -1907-05-05 12:34:56.1037,1907-12-25 -1907-05-05 12:34:56.1038,1907-12-25 -1907-05-05 12:34:56.1039,1907-12-25 -1907-05-05 12:34:56.104,1907-12-25 -1907-05-05 12:34:56.1041,1907-12-25 -1907-05-05 12:34:56.1042,1907-12-25 -1907-05-05 12:34:56.1043,1907-12-25 -1907-05-05 12:34:56.1044,1907-12-25 -1907-05-05 12:34:56.1045,1907-12-25 -1907-05-05 12:34:56.1046,1907-12-25 -1907-05-05 12:34:56.1047,1907-12-25 -1907-05-05 12:34:56.1048,1907-12-25 -1907-05-05 12:34:56.1049,1907-12-25 -1907-05-05 12:34:56.105,1907-12-25 -1907-05-05 12:34:56.1051,1907-12-25 -1907-05-05 12:34:56.1052,1907-12-25 -1907-05-05 12:34:56.1053,1907-12-25 -1907-05-05 12:34:56.1054,1907-12-25 -1907-05-05 12:34:56.1055,1907-12-25 -1907-05-05 12:34:56.1056,1907-12-25 -1907-05-05 12:34:56.1057,1907-12-25 -1907-05-05 12:34:56.1058,1907-12-25 -1907-05-05 12:34:56.1059,1907-12-25 -1907-05-05 12:34:56.106,1907-12-25 -1907-05-05 12:34:56.1061,1907-12-25 -1907-05-05 12:34:56.1062,1907-12-25 -1907-05-05 12:34:56.1063,1907-12-25 -1907-05-05 12:34:56.1064,1907-12-25 -1907-05-05 12:34:56.1065,1907-12-25 -1907-05-05 12:34:56.1066,1907-12-25 -1907-05-05 12:34:56.1067,1907-12-25 -1907-05-05 12:34:56.1068,1907-12-25 -1907-05-05 12:34:56.1069,1907-12-25 -1907-05-05 12:34:56.107,1907-12-25 -1907-05-05 12:34:56.1071,1907-12-25 -1907-05-05 12:34:56.1072,1907-12-25 -1907-05-05 12:34:56.1073,1907-12-25 -1907-05-05 12:34:56.1074,1907-12-25 -1907-05-05 12:34:56.1075,1907-12-25 -1907-05-05 12:34:56.1076,1907-12-25 -1907-05-05 12:34:56.1077,1907-12-25 -1907-05-05 12:34:56.1078,1907-12-25 -1907-05-05 12:34:56.1079,1907-12-25 -1907-05-05 12:34:56.108,1907-12-25 -1907-05-05 12:34:56.1081,1907-12-25 -1907-05-05 12:34:56.1082,1907-12-25 -1907-05-05 12:34:56.1083,1907-12-25 -1907-05-05 12:34:56.1084,1907-12-25 -1907-05-05 12:34:56.1085,1907-12-25 -1907-05-05 12:34:56.1086,1907-12-25 -1907-05-05 12:34:56.1087,1907-12-25 -1907-05-05 12:34:56.1088,1907-12-25 -1907-05-05 12:34:56.1089,1907-12-25 -1907-05-05 12:34:56.109,1907-12-25 -1907-05-05 12:34:56.1091,1907-12-25 -1907-05-05 12:34:56.1092,1907-12-25 -1907-05-05 12:34:56.1093,1907-12-25 -1907-05-05 12:34:56.1094,1907-12-25 -1907-05-05 12:34:56.1095,1907-12-25 -1907-05-05 12:34:56.1096,1907-12-25 -1907-05-05 12:34:56.1097,1907-12-25 -1907-05-05 12:34:56.1098,1907-12-25 -1907-05-05 12:34:56.1099,1907-12-25 -1907-05-05 12:34:56.11,1907-12-25 -1907-05-05 12:34:56.1101,1907-12-25 -1907-05-05 12:34:56.1102,1907-12-25 -1907-05-05 12:34:56.1103,1907-12-25 -1907-05-05 12:34:56.1104,1907-12-25 -1907-05-05 12:34:56.1105,1907-12-25 -1907-05-05 12:34:56.1106,1907-12-25 -1907-05-05 12:34:56.1107,1907-12-25 -1907-05-05 12:34:56.1108,1907-12-25 -1907-05-05 12:34:56.1109,1907-12-25 -1907-05-05 12:34:56.111,1907-12-25 -1907-05-05 12:34:56.1111,1907-12-25 -1907-05-05 12:34:56.1112,1907-12-25 -1907-05-05 12:34:56.1113,1907-12-25 -1907-05-05 12:34:56.1114,1907-12-25 -1907-05-05 12:34:56.1115,1907-12-25 -1907-05-05 12:34:56.1116,1907-12-25 -1907-05-05 12:34:56.1117,1907-12-25 -1907-05-05 12:34:56.1118,1907-12-25 -1907-05-05 12:34:56.1119,1907-12-25 -1907-05-05 12:34:56.112,1907-12-25 -1907-05-05 12:34:56.1121,1907-12-25 -1907-05-05 12:34:56.1122,1907-12-25 -1907-05-05 12:34:56.1123,1907-12-25 -1907-05-05 12:34:56.1124,1907-12-25 -1907-05-05 12:34:56.1125,1907-12-25 -1907-05-05 12:34:56.1126,1907-12-25 -1907-05-05 12:34:56.1127,1907-12-25 -1907-05-05 12:34:56.1128,1907-12-25 -1907-05-05 12:34:56.1129,1907-12-25 -1907-05-05 12:34:56.113,1907-12-25 -1907-05-05 12:34:56.1131,1907-12-25 -1907-05-05 12:34:56.1132,1907-12-25 -1907-05-05 12:34:56.1133,1907-12-25 -1907-05-05 12:34:56.1134,1907-12-25 -1907-05-05 12:34:56.1135,1907-12-25 -1907-05-05 12:34:56.1136,1907-12-25 -1907-05-05 12:34:56.1137,1907-12-25 -1907-05-05 12:34:56.1138,1907-12-25 -1907-05-05 12:34:56.1139,1907-12-25 -1907-05-05 12:34:56.114,1907-12-25 -1907-05-05 12:34:56.1141,1907-12-25 -1907-05-05 12:34:56.1142,1907-12-25 -1907-05-05 12:34:56.1143,1907-12-25 -1907-05-05 12:34:56.1144,1907-12-25 -1907-05-05 12:34:56.1145,1907-12-25 -1907-05-05 12:34:56.1146,1907-12-25 -1907-05-05 12:34:56.1147,1907-12-25 -1907-05-05 12:34:56.1148,1907-12-25 -1907-05-05 12:34:56.1149,1907-12-25 -1907-05-05 12:34:56.115,1907-12-25 -1907-05-05 12:34:56.1151,1907-12-25 -1907-05-05 12:34:56.1152,1907-12-25 -1907-05-05 12:34:56.1153,1907-12-25 -1907-05-05 12:34:56.1154,1907-12-25 -1907-05-05 12:34:56.1155,1907-12-25 -1907-05-05 12:34:56.1156,1907-12-25 -1907-05-05 12:34:56.1157,1907-12-25 -1907-05-05 12:34:56.1158,1907-12-25 -1907-05-05 12:34:56.1159,1907-12-25 -1907-05-05 12:34:56.116,1907-12-25 -1907-05-05 12:34:56.1161,1907-12-25 -1907-05-05 12:34:56.1162,1907-12-25 -1907-05-05 12:34:56.1163,1907-12-25 -1907-05-05 12:34:56.1164,1907-12-25 -1907-05-05 12:34:56.1165,1907-12-25 -1907-05-05 12:34:56.1166,1907-12-25 -1907-05-05 12:34:56.1167,1907-12-25 -1907-05-05 12:34:56.1168,1907-12-25 -1907-05-05 12:34:56.1169,1907-12-25 -1907-05-05 12:34:56.117,1907-12-25 -1907-05-05 12:34:56.1171,1907-12-25 -1907-05-05 12:34:56.1172,1907-12-25 -1907-05-05 12:34:56.1173,1907-12-25 -1907-05-05 12:34:56.1174,1907-12-25 -1907-05-05 12:34:56.1175,1907-12-25 -1907-05-05 12:34:56.1176,1907-12-25 -1907-05-05 12:34:56.1177,1907-12-25 -1907-05-05 12:34:56.1178,1907-12-25 -1907-05-05 12:34:56.1179,1907-12-25 -1907-05-05 12:34:56.118,1907-12-25 -1907-05-05 12:34:56.1181,1907-12-25 -1907-05-05 12:34:56.1182,1907-12-25 -1907-05-05 12:34:56.1183,1907-12-25 -1907-05-05 12:34:56.1184,1907-12-25 -1907-05-05 12:34:56.1185,1907-12-25 -1907-05-05 12:34:56.1186,1907-12-25 -1907-05-05 12:34:56.1187,1907-12-25 -1907-05-05 12:34:56.1188,1907-12-25 -1907-05-05 12:34:56.1189,1907-12-25 -1907-05-05 12:34:56.119,1907-12-25 -1907-05-05 12:34:56.1191,1907-12-25 -1907-05-05 12:34:56.1192,1907-12-25 -1907-05-05 12:34:56.1193,1907-12-25 -1907-05-05 12:34:56.1194,1907-12-25 -1907-05-05 12:34:56.1195,1907-12-25 -1907-05-05 12:34:56.1196,1907-12-25 -1907-05-05 12:34:56.1197,1907-12-25 -1907-05-05 12:34:56.1198,1907-12-25 -1907-05-05 12:34:56.1199,1907-12-25 -1907-05-05 12:34:56.12,1907-12-25 -1907-05-05 12:34:56.1201,1907-12-25 -1907-05-05 12:34:56.1202,1907-12-25 -1907-05-05 12:34:56.1203,1907-12-25 -1907-05-05 12:34:56.1204,1907-12-25 -1907-05-05 12:34:56.1205,1907-12-25 -1907-05-05 12:34:56.1206,1907-12-25 -1907-05-05 12:34:56.1207,1907-12-25 -1907-05-05 12:34:56.1208,1907-12-25 -1907-05-05 12:34:56.1209,1907-12-25 -1907-05-05 12:34:56.121,1907-12-25 -1907-05-05 12:34:56.1211,1907-12-25 -1907-05-05 12:34:56.1212,1907-12-25 -1907-05-05 12:34:56.1213,1907-12-25 -1907-05-05 12:34:56.1214,1907-12-25 -1907-05-05 12:34:56.1215,1907-12-25 -1907-05-05 12:34:56.1216,1907-12-25 -1907-05-05 12:34:56.1217,1907-12-25 -1907-05-05 12:34:56.1218,1907-12-25 -1907-05-05 12:34:56.1219,1907-12-25 -1907-05-05 12:34:56.122,1907-12-25 -1907-05-05 12:34:56.1221,1907-12-25 -1907-05-05 12:34:56.1222,1907-12-25 -1907-05-05 12:34:56.1223,1907-12-25 -1907-05-05 12:34:56.1224,1907-12-25 -1907-05-05 12:34:56.1225,1907-12-25 -1907-05-05 12:34:56.1226,1907-12-25 -1907-05-05 12:34:56.1227,1907-12-25 -1907-05-05 12:34:56.1228,1907-12-25 -1907-05-05 12:34:56.1229,1907-12-25 -1907-05-05 12:34:56.123,1907-12-25 -1907-05-05 12:34:56.1231,1907-12-25 -1907-05-05 12:34:56.1232,1907-12-25 -1907-05-05 12:34:56.1233,1907-12-25 -1907-05-05 12:34:56.1234,1907-12-25 -1907-05-05 12:34:56.1235,1907-12-25 -1907-05-05 12:34:56.1236,1907-12-25 -1907-05-05 12:34:56.1237,1907-12-25 -1907-05-05 12:34:56.1238,1907-12-25 -1907-05-05 12:34:56.1239,1907-12-25 -1907-05-05 12:34:56.124,1907-12-25 -1907-05-05 12:34:56.1241,1907-12-25 -1907-05-05 12:34:56.1242,1907-12-25 -1907-05-05 12:34:56.1243,1907-12-25 -1907-05-05 12:34:56.1244,1907-12-25 -1907-05-05 12:34:56.1245,1907-12-25 -1907-05-05 12:34:56.1246,1907-12-25 -1907-05-05 12:34:56.1247,1907-12-25 -1907-05-05 12:34:56.1248,1907-12-25 -1907-05-05 12:34:56.1249,1907-12-25 -1907-05-05 12:34:56.125,1907-12-25 -1907-05-05 12:34:56.1251,1907-12-25 -1907-05-05 12:34:56.1252,1907-12-25 -1907-05-05 12:34:56.1253,1907-12-25 -1907-05-05 12:34:56.1254,1907-12-25 -1907-05-05 12:34:56.1255,1907-12-25 -1907-05-05 12:34:56.1256,1907-12-25 -1907-05-05 12:34:56.1257,1907-12-25 -1907-05-05 12:34:56.1258,1907-12-25 -1907-05-05 12:34:56.1259,1907-12-25 -1907-05-05 12:34:56.126,1907-12-25 -1907-05-05 12:34:56.1261,1907-12-25 -1907-05-05 12:34:56.1262,1907-12-25 -1907-05-05 12:34:56.1263,1907-12-25 -1907-05-05 12:34:56.1264,1907-12-25 -1907-05-05 12:34:56.1265,1907-12-25 -1907-05-05 12:34:56.1266,1907-12-25 -1907-05-05 12:34:56.1267,1907-12-25 -1907-05-05 12:34:56.1268,1907-12-25 -1907-05-05 12:34:56.1269,1907-12-25 -1907-05-05 12:34:56.127,1907-12-25 -1907-05-05 12:34:56.1271,1907-12-25 -1907-05-05 12:34:56.1272,1907-12-25 -1907-05-05 12:34:56.1273,1907-12-25 -1907-05-05 12:34:56.1274,1907-12-25 -1907-05-05 12:34:56.1275,1907-12-25 -1907-05-05 12:34:56.1276,1907-12-25 -1907-05-05 12:34:56.1277,1907-12-25 -1907-05-05 12:34:56.1278,1907-12-25 -1907-05-05 12:34:56.1279,1907-12-25 -1907-05-05 12:34:56.128,1907-12-25 -1907-05-05 12:34:56.1281,1907-12-25 -1907-05-05 12:34:56.1282,1907-12-25 -1907-05-05 12:34:56.1283,1907-12-25 -1907-05-05 12:34:56.1284,1907-12-25 -1907-05-05 12:34:56.1285,1907-12-25 -1907-05-05 12:34:56.1286,1907-12-25 -1907-05-05 12:34:56.1287,1907-12-25 -1907-05-05 12:34:56.1288,1907-12-25 -1907-05-05 12:34:56.1289,1907-12-25 -1907-05-05 12:34:56.129,1907-12-25 -1907-05-05 12:34:56.1291,1907-12-25 -1907-05-05 12:34:56.1292,1907-12-25 -1907-05-05 12:34:56.1293,1907-12-25 -1907-05-05 12:34:56.1294,1907-12-25 -1907-05-05 12:34:56.1295,1907-12-25 -1907-05-05 12:34:56.1296,1907-12-25 -1907-05-05 12:34:56.1297,1907-12-25 -1907-05-05 12:34:56.1298,1907-12-25 -1907-05-05 12:34:56.1299,1907-12-25 -1907-05-05 12:34:56.13,1907-12-25 -1907-05-05 12:34:56.1301,1907-12-25 -1907-05-05 12:34:56.1302,1907-12-25 -1907-05-05 12:34:56.1303,1907-12-25 -1907-05-05 12:34:56.1304,1907-12-25 -1907-05-05 12:34:56.1305,1907-12-25 -1907-05-05 12:34:56.1306,1907-12-25 -1907-05-05 12:34:56.1307,1907-12-25 -1907-05-05 12:34:56.1308,1907-12-25 -1907-05-05 12:34:56.1309,1907-12-25 -1907-05-05 12:34:56.131,1907-12-25 -1907-05-05 12:34:56.1311,1907-12-25 -1907-05-05 12:34:56.1312,1907-12-25 -1907-05-05 12:34:56.1313,1907-12-25 -1907-05-05 12:34:56.1314,1907-12-25 -1907-05-05 12:34:56.1315,1907-12-25 -1907-05-05 12:34:56.1316,1907-12-25 -1907-05-05 12:34:56.1317,1907-12-25 -1907-05-05 12:34:56.1318,1907-12-25 -1907-05-05 12:34:56.1319,1907-12-25 -1907-05-05 12:34:56.132,1907-12-25 -1907-05-05 12:34:56.1321,1907-12-25 -1907-05-05 12:34:56.1322,1907-12-25 -1907-05-05 12:34:56.1323,1907-12-25 -1907-05-05 12:34:56.1324,1907-12-25 -1907-05-05 12:34:56.1325,1907-12-25 -1907-05-05 12:34:56.1326,1907-12-25 -1907-05-05 12:34:56.1327,1907-12-25 -1907-05-05 12:34:56.1328,1907-12-25 -1907-05-05 12:34:56.1329,1907-12-25 -1907-05-05 12:34:56.133,1907-12-25 -1907-05-05 12:34:56.1331,1907-12-25 -1907-05-05 12:34:56.1332,1907-12-25 -1907-05-05 12:34:56.1333,1907-12-25 -1907-05-05 12:34:56.1334,1907-12-25 -1907-05-05 12:34:56.1335,1907-12-25 -1907-05-05 12:34:56.1336,1907-12-25 -1907-05-05 12:34:56.1337,1907-12-25 -1907-05-05 12:34:56.1338,1907-12-25 -1907-05-05 12:34:56.1339,1907-12-25 -1907-05-05 12:34:56.134,1907-12-25 -1907-05-05 12:34:56.1341,1907-12-25 -1907-05-05 12:34:56.1342,1907-12-25 -1907-05-05 12:34:56.1343,1907-12-25 -1907-05-05 12:34:56.1344,1907-12-25 -1907-05-05 12:34:56.1345,1907-12-25 -1907-05-05 12:34:56.1346,1907-12-25 -1907-05-05 12:34:56.1347,1907-12-25 -1907-05-05 12:34:56.1348,1907-12-25 -1907-05-05 12:34:56.1349,1907-12-25 -1907-05-05 12:34:56.135,1907-12-25 -1907-05-05 12:34:56.1351,1907-12-25 -1907-05-05 12:34:56.1352,1907-12-25 -1907-05-05 12:34:56.1353,1907-12-25 -1907-05-05 12:34:56.1354,1907-12-25 -1907-05-05 12:34:56.1355,1907-12-25 -1907-05-05 12:34:56.1356,1907-12-25 -1907-05-05 12:34:56.1357,1907-12-25 -1907-05-05 12:34:56.1358,1907-12-25 -1907-05-05 12:34:56.1359,1907-12-25 -1907-05-05 12:34:56.136,1907-12-25 -1907-05-05 12:34:56.1361,1907-12-25 -1907-05-05 12:34:56.1362,1907-12-25 -1907-05-05 12:34:56.1363,1907-12-25 -1907-05-05 12:34:56.1364,1907-12-25 -1907-05-05 12:34:56.1365,1907-12-25 -1907-05-05 12:34:56.1366,1907-12-25 -1907-05-05 12:34:56.1367,1907-12-25 -1907-05-05 12:34:56.1368,1907-12-25 -1907-05-05 12:34:56.1369,1907-12-25 -1907-05-05 12:34:56.137,1907-12-25 -1907-05-05 12:34:56.1371,1907-12-25 -1907-05-05 12:34:56.1372,1907-12-25 -1907-05-05 12:34:56.1373,1907-12-25 -1907-05-05 12:34:56.1374,1907-12-25 -1907-05-05 12:34:56.1375,1907-12-25 -1907-05-05 12:34:56.1376,1907-12-25 -1907-05-05 12:34:56.1377,1907-12-25 -1907-05-05 12:34:56.1378,1907-12-25 -1907-05-05 12:34:56.1379,1907-12-25 -1907-05-05 12:34:56.138,1907-12-25 -1907-05-05 12:34:56.1381,1907-12-25 -1907-05-05 12:34:56.1382,1907-12-25 -1907-05-05 12:34:56.1383,1907-12-25 -1907-05-05 12:34:56.1384,1907-12-25 -1907-05-05 12:34:56.1385,1907-12-25 -1907-05-05 12:34:56.1386,1907-12-25 -1907-05-05 12:34:56.1387,1907-12-25 -1907-05-05 12:34:56.1388,1907-12-25 -1907-05-05 12:34:56.1389,1907-12-25 -1907-05-05 12:34:56.139,1907-12-25 -1907-05-05 12:34:56.1391,1907-12-25 -1907-05-05 12:34:56.1392,1907-12-25 -1907-05-05 12:34:56.1393,1907-12-25 -1907-05-05 12:34:56.1394,1907-12-25 -1907-05-05 12:34:56.1395,1907-12-25 -1907-05-05 12:34:56.1396,1907-12-25 -1907-05-05 12:34:56.1397,1907-12-25 -1907-05-05 12:34:56.1398,1907-12-25 -1907-05-05 12:34:56.1399,1907-12-25 -1907-05-05 12:34:56.14,1907-12-25 -1907-05-05 12:34:56.1401,1907-12-25 -1907-05-05 12:34:56.1402,1907-12-25 -1907-05-05 12:34:56.1403,1907-12-25 -1907-05-05 12:34:56.1404,1907-12-25 -1907-05-05 12:34:56.1405,1907-12-25 -1907-05-05 12:34:56.1406,1907-12-25 -1907-05-05 12:34:56.1407,1907-12-25 -1907-05-05 12:34:56.1408,1907-12-25 -1907-05-05 12:34:56.1409,1907-12-25 -1907-05-05 12:34:56.141,1907-12-25 -1907-05-05 12:34:56.1411,1907-12-25 -1907-05-05 12:34:56.1412,1907-12-25 -1907-05-05 12:34:56.1413,1907-12-25 -1907-05-05 12:34:56.1414,1907-12-25 -1907-05-05 12:34:56.1415,1907-12-25 -1907-05-05 12:34:56.1416,1907-12-25 -1907-05-05 12:34:56.1417,1907-12-25 -1907-05-05 12:34:56.1418,1907-12-25 -1907-05-05 12:34:56.1419,1907-12-25 -1907-05-05 12:34:56.142,1907-12-25 -1907-05-05 12:34:56.1421,1907-12-25 -1907-05-05 12:34:56.1422,1907-12-25 -1907-05-05 12:34:56.1423,1907-12-25 -1907-05-05 12:34:56.1424,1907-12-25 -1907-05-05 12:34:56.1425,1907-12-25 -1907-05-05 12:34:56.1426,1907-12-25 -1907-05-05 12:34:56.1427,1907-12-25 -1907-05-05 12:34:56.1428,1907-12-25 -1907-05-05 12:34:56.1429,1907-12-25 -1907-05-05 12:34:56.143,1907-12-25 -1907-05-05 12:34:56.1431,1907-12-25 -1907-05-05 12:34:56.1432,1907-12-25 -1907-05-05 12:34:56.1433,1907-12-25 -1907-05-05 12:34:56.1434,1907-12-25 -1907-05-05 12:34:56.1435,1907-12-25 -1907-05-05 12:34:56.1436,1907-12-25 -1907-05-05 12:34:56.1437,1907-12-25 -1907-05-05 12:34:56.1438,1907-12-25 -1907-05-05 12:34:56.1439,1907-12-25 -1907-05-05 12:34:56.144,1907-12-25 -1907-05-05 12:34:56.1441,1907-12-25 -1907-05-05 12:34:56.1442,1907-12-25 -1907-05-05 12:34:56.1443,1907-12-25 -1907-05-05 12:34:56.1444,1907-12-25 -1907-05-05 12:34:56.1445,1907-12-25 -1907-05-05 12:34:56.1446,1907-12-25 -1907-05-05 12:34:56.1447,1907-12-25 -1907-05-05 12:34:56.1448,1907-12-25 -1907-05-05 12:34:56.1449,1907-12-25 -1907-05-05 12:34:56.145,1907-12-25 -1907-05-05 12:34:56.1451,1907-12-25 -1907-05-05 12:34:56.1452,1907-12-25 -1907-05-05 12:34:56.1453,1907-12-25 -1907-05-05 12:34:56.1454,1907-12-25 -1907-05-05 12:34:56.1455,1907-12-25 -1907-05-05 12:34:56.1456,1907-12-25 -1907-05-05 12:34:56.1457,1907-12-25 -1907-05-05 12:34:56.1458,1907-12-25 -1907-05-05 12:34:56.1459,1907-12-25 -1907-05-05 12:34:56.146,1907-12-25 -1907-05-05 12:34:56.1461,1907-12-25 -1907-05-05 12:34:56.1462,1907-12-25 -1907-05-05 12:34:56.1463,1907-12-25 -1907-05-05 12:34:56.1464,1907-12-25 -1907-05-05 12:34:56.1465,1907-12-25 -1907-05-05 12:34:56.1466,1907-12-25 -1907-05-05 12:34:56.1467,1907-12-25 -1907-05-05 12:34:56.1468,1907-12-25 -1907-05-05 12:34:56.1469,1907-12-25 -1907-05-05 12:34:56.147,1907-12-25 -1907-05-05 12:34:56.1471,1907-12-25 -1907-05-05 12:34:56.1472,1907-12-25 -1907-05-05 12:34:56.1473,1907-12-25 -1907-05-05 12:34:56.1474,1907-12-25 -1907-05-05 12:34:56.1475,1907-12-25 -1907-05-05 12:34:56.1476,1907-12-25 -1907-05-05 12:34:56.1477,1907-12-25 -1907-05-05 12:34:56.1478,1907-12-25 -1907-05-05 12:34:56.1479,1907-12-25 -1907-05-05 12:34:56.148,1907-12-25 -1907-05-05 12:34:56.1481,1907-12-25 -1907-05-05 12:34:56.1482,1907-12-25 -1907-05-05 12:34:56.1483,1907-12-25 -1907-05-05 12:34:56.1484,1907-12-25 -1907-05-05 12:34:56.1485,1907-12-25 -1907-05-05 12:34:56.1486,1907-12-25 -1907-05-05 12:34:56.1487,1907-12-25 -1907-05-05 12:34:56.1488,1907-12-25 -1907-05-05 12:34:56.1489,1907-12-25 -1907-05-05 12:34:56.149,1907-12-25 -1907-05-05 12:34:56.1491,1907-12-25 -1907-05-05 12:34:56.1492,1907-12-25 -1907-05-05 12:34:56.1493,1907-12-25 -1907-05-05 12:34:56.1494,1907-12-25 -1907-05-05 12:34:56.1495,1907-12-25 -1907-05-05 12:34:56.1496,1907-12-25 -1907-05-05 12:34:56.1497,1907-12-25 -1907-05-05 12:34:56.1498,1907-12-25 -1907-05-05 12:34:56.1499,1907-12-25 -1907-05-05 12:34:56.15,1907-12-25 -1907-05-05 12:34:56.1501,1907-12-25 -1907-05-05 12:34:56.1502,1907-12-25 -1907-05-05 12:34:56.1503,1907-12-25 -1907-05-05 12:34:56.1504,1907-12-25 -1907-05-05 12:34:56.1505,1907-12-25 -1907-05-05 12:34:56.1506,1907-12-25 -1907-05-05 12:34:56.1507,1907-12-25 -1907-05-05 12:34:56.1508,1907-12-25 -1907-05-05 12:34:56.1509,1907-12-25 -1907-05-05 12:34:56.151,1907-12-25 -1907-05-05 12:34:56.1511,1907-12-25 -1907-05-05 12:34:56.1512,1907-12-25 -1907-05-05 12:34:56.1513,1907-12-25 -1907-05-05 12:34:56.1514,1907-12-25 -1907-05-05 12:34:56.1515,1907-12-25 -1907-05-05 12:34:56.1516,1907-12-25 -1907-05-05 12:34:56.1517,1907-12-25 -1907-05-05 12:34:56.1518,1907-12-25 -1907-05-05 12:34:56.1519,1907-12-25 -1907-05-05 12:34:56.152,1907-12-25 -1907-05-05 12:34:56.1521,1907-12-25 -1907-05-05 12:34:56.1522,1907-12-25 -1907-05-05 12:34:56.1523,1907-12-25 -1907-05-05 12:34:56.1524,1907-12-25 -1907-05-05 12:34:56.1525,1907-12-25 -1907-05-05 12:34:56.1526,1907-12-25 -1907-05-05 12:34:56.1527,1907-12-25 -1907-05-05 12:34:56.1528,1907-12-25 -1907-05-05 12:34:56.1529,1907-12-25 -1907-05-05 12:34:56.153,1907-12-25 -1907-05-05 12:34:56.1531,1907-12-25 -1907-05-05 12:34:56.1532,1907-12-25 -1907-05-05 12:34:56.1533,1907-12-25 -1907-05-05 12:34:56.1534,1907-12-25 -1907-05-05 12:34:56.1535,1907-12-25 -1907-05-05 12:34:56.1536,1907-12-25 -1907-05-05 12:34:56.1537,1907-12-25 -1907-05-05 12:34:56.1538,1907-12-25 -1907-05-05 12:34:56.1539,1907-12-25 -1907-05-05 12:34:56.154,1907-12-25 -1907-05-05 12:34:56.1541,1907-12-25 -1907-05-05 12:34:56.1542,1907-12-25 -1907-05-05 12:34:56.1543,1907-12-25 -1907-05-05 12:34:56.1544,1907-12-25 -1907-05-05 12:34:56.1545,1907-12-25 -1907-05-05 12:34:56.1546,1907-12-25 -1907-05-05 12:34:56.1547,1907-12-25 -1907-05-05 12:34:56.1548,1907-12-25 -1907-05-05 12:34:56.1549,1907-12-25 -1907-05-05 12:34:56.155,1907-12-25 -1907-05-05 12:34:56.1551,1907-12-25 -1907-05-05 12:34:56.1552,1907-12-25 -1907-05-05 12:34:56.1553,1907-12-25 -1907-05-05 12:34:56.1554,1907-12-25 -1907-05-05 12:34:56.1555,1907-12-25 -1907-05-05 12:34:56.1556,1907-12-25 -1907-05-05 12:34:56.1557,1907-12-25 -1907-05-05 12:34:56.1558,1907-12-25 -1907-05-05 12:34:56.1559,1907-12-25 -1907-05-05 12:34:56.156,1907-12-25 -1907-05-05 12:34:56.1561,1907-12-25 -1907-05-05 12:34:56.1562,1907-12-25 -1907-05-05 12:34:56.1563,1907-12-25 -1907-05-05 12:34:56.1564,1907-12-25 -1907-05-05 12:34:56.1565,1907-12-25 -1907-05-05 12:34:56.1566,1907-12-25 -1907-05-05 12:34:56.1567,1907-12-25 -1907-05-05 12:34:56.1568,1907-12-25 -1907-05-05 12:34:56.1569,1907-12-25 -1907-05-05 12:34:56.157,1907-12-25 -1907-05-05 12:34:56.1571,1907-12-25 -1907-05-05 12:34:56.1572,1907-12-25 -1907-05-05 12:34:56.1573,1907-12-25 -1907-05-05 12:34:56.1574,1907-12-25 -1907-05-05 12:34:56.1575,1907-12-25 -1907-05-05 12:34:56.1576,1907-12-25 -1907-05-05 12:34:56.1577,1907-12-25 -1907-05-05 12:34:56.1578,1907-12-25 -1907-05-05 12:34:56.1579,1907-12-25 -1907-05-05 12:34:56.158,1907-12-25 -1907-05-05 12:34:56.1581,1907-12-25 -1907-05-05 12:34:56.1582,1907-12-25 -1907-05-05 12:34:56.1583,1907-12-25 -1907-05-05 12:34:56.1584,1907-12-25 -1907-05-05 12:34:56.1585,1907-12-25 -1907-05-05 12:34:56.1586,1907-12-25 -1907-05-05 12:34:56.1587,1907-12-25 -1907-05-05 12:34:56.1588,1907-12-25 -1907-05-05 12:34:56.1589,1907-12-25 -1907-05-05 12:34:56.159,1907-12-25 -1907-05-05 12:34:56.1591,1907-12-25 -1907-05-05 12:34:56.1592,1907-12-25 -1907-05-05 12:34:56.1593,1907-12-25 -1907-05-05 12:34:56.1594,1907-12-25 -1907-05-05 12:34:56.1595,1907-12-25 -1907-05-05 12:34:56.1596,1907-12-25 -1907-05-05 12:34:56.1597,1907-12-25 -1907-05-05 12:34:56.1598,1907-12-25 -1907-05-05 12:34:56.1599,1907-12-25 -1907-05-05 12:34:56.16,1907-12-25 -1907-05-05 12:34:56.1601,1907-12-25 -1907-05-05 12:34:56.1602,1907-12-25 -1907-05-05 12:34:56.1603,1907-12-25 -1907-05-05 12:34:56.1604,1907-12-25 -1907-05-05 12:34:56.1605,1907-12-25 -1907-05-05 12:34:56.1606,1907-12-25 -1907-05-05 12:34:56.1607,1907-12-25 -1907-05-05 12:34:56.1608,1907-12-25 -1907-05-05 12:34:56.1609,1907-12-25 -1907-05-05 12:34:56.161,1907-12-25 -1907-05-05 12:34:56.1611,1907-12-25 -1907-05-05 12:34:56.1612,1907-12-25 -1907-05-05 12:34:56.1613,1907-12-25 -1907-05-05 12:34:56.1614,1907-12-25 -1907-05-05 12:34:56.1615,1907-12-25 -1907-05-05 12:34:56.1616,1907-12-25 -1907-05-05 12:34:56.1617,1907-12-25 -1907-05-05 12:34:56.1618,1907-12-25 -1907-05-05 12:34:56.1619,1907-12-25 -1907-05-05 12:34:56.162,1907-12-25 -1907-05-05 12:34:56.1621,1907-12-25 -1907-05-05 12:34:56.1622,1907-12-25 -1907-05-05 12:34:56.1623,1907-12-25 -1907-05-05 12:34:56.1624,1907-12-25 -1907-05-05 12:34:56.1625,1907-12-25 -1907-05-05 12:34:56.1626,1907-12-25 -1907-05-05 12:34:56.1627,1907-12-25 -1907-05-05 12:34:56.1628,1907-12-25 -1907-05-05 12:34:56.1629,1907-12-25 -1907-05-05 12:34:56.163,1907-12-25 -1907-05-05 12:34:56.1631,1907-12-25 -1907-05-05 12:34:56.1632,1907-12-25 -1907-05-05 12:34:56.1633,1907-12-25 -1907-05-05 12:34:56.1634,1907-12-25 -1907-05-05 12:34:56.1635,1907-12-25 -1907-05-05 12:34:56.1636,1907-12-25 -1907-05-05 12:34:56.1637,1907-12-25 -1907-05-05 12:34:56.1638,1907-12-25 -1907-05-05 12:34:56.1639,1907-12-25 -1907-05-05 12:34:56.164,1907-12-25 -1907-05-05 12:34:56.1641,1907-12-25 -1907-05-05 12:34:56.1642,1907-12-25 -1907-05-05 12:34:56.1643,1907-12-25 -1907-05-05 12:34:56.1644,1907-12-25 -1907-05-05 12:34:56.1645,1907-12-25 -1907-05-05 12:34:56.1646,1907-12-25 -1907-05-05 12:34:56.1647,1907-12-25 -1907-05-05 12:34:56.1648,1907-12-25 -1907-05-05 12:34:56.1649,1907-12-25 -1907-05-05 12:34:56.165,1907-12-25 -1907-05-05 12:34:56.1651,1907-12-25 -1907-05-05 12:34:56.1652,1907-12-25 -1907-05-05 12:34:56.1653,1907-12-25 -1907-05-05 12:34:56.1654,1907-12-25 -1907-05-05 12:34:56.1655,1907-12-25 -1907-05-05 12:34:56.1656,1907-12-25 -1907-05-05 12:34:56.1657,1907-12-25 -1907-05-05 12:34:56.1658,1907-12-25 -1907-05-05 12:34:56.1659,1907-12-25 -1907-05-05 12:34:56.166,1907-12-25 -1907-05-05 12:34:56.1661,1907-12-25 -1907-05-05 12:34:56.1662,1907-12-25 -1907-05-05 12:34:56.1663,1907-12-25 -1907-05-05 12:34:56.1664,1907-12-25 -1907-05-05 12:34:56.1665,1907-12-25 -1907-05-05 12:34:56.1666,1907-12-25 -1907-05-05 12:34:56.1667,1907-12-25 -1907-05-05 12:34:56.1668,1907-12-25 -1907-05-05 12:34:56.1669,1907-12-25 -1907-05-05 12:34:56.167,1907-12-25 -1907-05-05 12:34:56.1671,1907-12-25 -1907-05-05 12:34:56.1672,1907-12-25 -1907-05-05 12:34:56.1673,1907-12-25 -1907-05-05 12:34:56.1674,1907-12-25 -1907-05-05 12:34:56.1675,1907-12-25 -1907-05-05 12:34:56.1676,1907-12-25 -1907-05-05 12:34:56.1677,1907-12-25 -1907-05-05 12:34:56.1678,1907-12-25 -1907-05-05 12:34:56.1679,1907-12-25 -1907-05-05 12:34:56.168,1907-12-25 -1907-05-05 12:34:56.1681,1907-12-25 -1907-05-05 12:34:56.1682,1907-12-25 -1907-05-05 12:34:56.1683,1907-12-25 -1907-05-05 12:34:56.1684,1907-12-25 -1907-05-05 12:34:56.1685,1907-12-25 -1907-05-05 12:34:56.1686,1907-12-25 -1907-05-05 12:34:56.1687,1907-12-25 -1907-05-05 12:34:56.1688,1907-12-25 -1907-05-05 12:34:56.1689,1907-12-25 -1907-05-05 12:34:56.169,1907-12-25 -1907-05-05 12:34:56.1691,1907-12-25 -1907-05-05 12:34:56.1692,1907-12-25 -1907-05-05 12:34:56.1693,1907-12-25 -1907-05-05 12:34:56.1694,1907-12-25 -1907-05-05 12:34:56.1695,1907-12-25 -1907-05-05 12:34:56.1696,1907-12-25 -1907-05-05 12:34:56.1697,1907-12-25 -1907-05-05 12:34:56.1698,1907-12-25 -1907-05-05 12:34:56.1699,1907-12-25 -1907-05-05 12:34:56.17,1907-12-25 -1907-05-05 12:34:56.1701,1907-12-25 -1907-05-05 12:34:56.1702,1907-12-25 -1907-05-05 12:34:56.1703,1907-12-25 -1907-05-05 12:34:56.1704,1907-12-25 -1907-05-05 12:34:56.1705,1907-12-25 -1907-05-05 12:34:56.1706,1907-12-25 -1907-05-05 12:34:56.1707,1907-12-25 -1907-05-05 12:34:56.1708,1907-12-25 -1907-05-05 12:34:56.1709,1907-12-25 -1907-05-05 12:34:56.171,1907-12-25 -1907-05-05 12:34:56.1711,1907-12-25 -1907-05-05 12:34:56.1712,1907-12-25 -1907-05-05 12:34:56.1713,1907-12-25 -1907-05-05 12:34:56.1714,1907-12-25 -1907-05-05 12:34:56.1715,1907-12-25 -1907-05-05 12:34:56.1716,1907-12-25 -1907-05-05 12:34:56.1717,1907-12-25 -1907-05-05 12:34:56.1718,1907-12-25 -1907-05-05 12:34:56.1719,1907-12-25 -1907-05-05 12:34:56.172,1907-12-25 -1907-05-05 12:34:56.1721,1907-12-25 -1907-05-05 12:34:56.1722,1907-12-25 -1907-05-05 12:34:56.1723,1907-12-25 -1907-05-05 12:34:56.1724,1907-12-25 -1907-05-05 12:34:56.1725,1907-12-25 -1907-05-05 12:34:56.1726,1907-12-25 -1907-05-05 12:34:56.1727,1907-12-25 -1907-05-05 12:34:56.1728,1907-12-25 -1907-05-05 12:34:56.1729,1907-12-25 -1907-05-05 12:34:56.173,1907-12-25 -1907-05-05 12:34:56.1731,1907-12-25 -1907-05-05 12:34:56.1732,1907-12-25 -1907-05-05 12:34:56.1733,1907-12-25 -1907-05-05 12:34:56.1734,1907-12-25 -1907-05-05 12:34:56.1735,1907-12-25 -1907-05-05 12:34:56.1736,1907-12-25 -1907-05-05 12:34:56.1737,1907-12-25 -1907-05-05 12:34:56.1738,1907-12-25 -1907-05-05 12:34:56.1739,1907-12-25 -1907-05-05 12:34:56.174,1907-12-25 -1907-05-05 12:34:56.1741,1907-12-25 -1907-05-05 12:34:56.1742,1907-12-25 -1907-05-05 12:34:56.1743,1907-12-25 -1907-05-05 12:34:56.1744,1907-12-25 -1907-05-05 12:34:56.1745,1907-12-25 -1907-05-05 12:34:56.1746,1907-12-25 -1907-05-05 12:34:56.1747,1907-12-25 -1907-05-05 12:34:56.1748,1907-12-25 -1907-05-05 12:34:56.1749,1907-12-25 -1907-05-05 12:34:56.175,1907-12-25 -1907-05-05 12:34:56.1751,1907-12-25 -1907-05-05 12:34:56.1752,1907-12-25 -1907-05-05 12:34:56.1753,1907-12-25 -1907-05-05 12:34:56.1754,1907-12-25 -1907-05-05 12:34:56.1755,1907-12-25 -1907-05-05 12:34:56.1756,1907-12-25 -1907-05-05 12:34:56.1757,1907-12-25 -1907-05-05 12:34:56.1758,1907-12-25 -1907-05-05 12:34:56.1759,1907-12-25 -1907-05-05 12:34:56.176,1907-12-25 -1907-05-05 12:34:56.1761,1907-12-25 -1907-05-05 12:34:56.1762,1907-12-25 -1907-05-05 12:34:56.1763,1907-12-25 -1907-05-05 12:34:56.1764,1907-12-25 -1907-05-05 12:34:56.1765,1907-12-25 -1907-05-05 12:34:56.1766,1907-12-25 -1907-05-05 12:34:56.1767,1907-12-25 -1907-05-05 12:34:56.1768,1907-12-25 -1907-05-05 12:34:56.1769,1907-12-25 -1907-05-05 12:34:56.177,1907-12-25 -1907-05-05 12:34:56.1771,1907-12-25 -1907-05-05 12:34:56.1772,1907-12-25 -1907-05-05 12:34:56.1773,1907-12-25 -1907-05-05 12:34:56.1774,1907-12-25 -1907-05-05 12:34:56.1775,1907-12-25 -1907-05-05 12:34:56.1776,1907-12-25 -1907-05-05 12:34:56.1777,1907-12-25 -1907-05-05 12:34:56.1778,1907-12-25 -1907-05-05 12:34:56.1779,1907-12-25 -1907-05-05 12:34:56.178,1907-12-25 -1907-05-05 12:34:56.1781,1907-12-25 -1907-05-05 12:34:56.1782,1907-12-25 -1907-05-05 12:34:56.1783,1907-12-25 -1907-05-05 12:34:56.1784,1907-12-25 -1907-05-05 12:34:56.1785,1907-12-25 -1907-05-05 12:34:56.1786,1907-12-25 -1907-05-05 12:34:56.1787,1907-12-25 -1907-05-05 12:34:56.1788,1907-12-25 -1907-05-05 12:34:56.1789,1907-12-25 -1907-05-05 12:34:56.179,1907-12-25 -1907-05-05 12:34:56.1791,1907-12-25 -1907-05-05 12:34:56.1792,1907-12-25 -1907-05-05 12:34:56.1793,1907-12-25 -1907-05-05 12:34:56.1794,1907-12-25 -1907-05-05 12:34:56.1795,1907-12-25 -1907-05-05 12:34:56.1796,1907-12-25 -1907-05-05 12:34:56.1797,1907-12-25 -1907-05-05 12:34:56.1798,1907-12-25 -1907-05-05 12:34:56.1799,1907-12-25 -1907-05-05 12:34:56.18,1907-12-25 -1907-05-05 12:34:56.1801,1907-12-25 -1907-05-05 12:34:56.1802,1907-12-25 -1907-05-05 12:34:56.1803,1907-12-25 -1907-05-05 12:34:56.1804,1907-12-25 -1907-05-05 12:34:56.1805,1907-12-25 -1907-05-05 12:34:56.1806,1907-12-25 -1907-05-05 12:34:56.1807,1907-12-25 -1907-05-05 12:34:56.1808,1907-12-25 -1907-05-05 12:34:56.1809,1907-12-25 -1907-05-05 12:34:56.181,1907-12-25 -1907-05-05 12:34:56.1811,1907-12-25 -1907-05-05 12:34:56.1812,1907-12-25 -1907-05-05 12:34:56.1813,1907-12-25 -1907-05-05 12:34:56.1814,1907-12-25 -1907-05-05 12:34:56.1815,1907-12-25 -1907-05-05 12:34:56.1816,1907-12-25 -1907-05-05 12:34:56.1817,1907-12-25 -1907-05-05 12:34:56.1818,1907-12-25 -1907-05-05 12:34:56.1819,1907-12-25 -1907-05-05 12:34:56.182,1907-12-25 -1907-05-05 12:34:56.1821,1907-12-25 -1907-05-05 12:34:56.1822,1907-12-25 -1907-05-05 12:34:56.1823,1907-12-25 -1907-05-05 12:34:56.1824,1907-12-25 -1907-05-05 12:34:56.1825,1907-12-25 -1907-05-05 12:34:56.1826,1907-12-25 -1907-05-05 12:34:56.1827,1907-12-25 -1907-05-05 12:34:56.1828,1907-12-25 -1907-05-05 12:34:56.1829,1907-12-25 -1907-05-05 12:34:56.183,1907-12-25 -1907-05-05 12:34:56.1831,1907-12-25 -1907-05-05 12:34:56.1832,1907-12-25 -1907-05-05 12:34:56.1833,1907-12-25 -1907-05-05 12:34:56.1834,1907-12-25 -1907-05-05 12:34:56.1835,1907-12-25 -1907-05-05 12:34:56.1836,1907-12-25 -1907-05-05 12:34:56.1837,1907-12-25 -1907-05-05 12:34:56.1838,1907-12-25 -1907-05-05 12:34:56.1839,1907-12-25 -1907-05-05 12:34:56.184,1907-12-25 -1907-05-05 12:34:56.1841,1907-12-25 -1907-05-05 12:34:56.1842,1907-12-25 -1907-05-05 12:34:56.1843,1907-12-25 -1907-05-05 12:34:56.1844,1907-12-25 -1907-05-05 12:34:56.1845,1907-12-25 -1907-05-05 12:34:56.1846,1907-12-25 -1907-05-05 12:34:56.1847,1907-12-25 -1907-05-05 12:34:56.1848,1907-12-25 -1907-05-05 12:34:56.1849,1907-12-25 -1907-05-05 12:34:56.185,1907-12-25 -1907-05-05 12:34:56.1851,1907-12-25 -1907-05-05 12:34:56.1852,1907-12-25 -1907-05-05 12:34:56.1853,1907-12-25 -1907-05-05 12:34:56.1854,1907-12-25 -1907-05-05 12:34:56.1855,1907-12-25 -1907-05-05 12:34:56.1856,1907-12-25 -1907-05-05 12:34:56.1857,1907-12-25 -1907-05-05 12:34:56.1858,1907-12-25 -1907-05-05 12:34:56.1859,1907-12-25 -1907-05-05 12:34:56.186,1907-12-25 -1907-05-05 12:34:56.1861,1907-12-25 -1907-05-05 12:34:56.1862,1907-12-25 -1907-05-05 12:34:56.1863,1907-12-25 -1907-05-05 12:34:56.1864,1907-12-25 -1907-05-05 12:34:56.1865,1907-12-25 -1907-05-05 12:34:56.1866,1907-12-25 -1907-05-05 12:34:56.1867,1907-12-25 -1907-05-05 12:34:56.1868,1907-12-25 -1907-05-05 12:34:56.1869,1907-12-25 -1907-05-05 12:34:56.187,1907-12-25 -1907-05-05 12:34:56.1871,1907-12-25 -1907-05-05 12:34:56.1872,1907-12-25 -1907-05-05 12:34:56.1873,1907-12-25 -1907-05-05 12:34:56.1874,1907-12-25 -1907-05-05 12:34:56.1875,1907-12-25 -1907-05-05 12:34:56.1876,1907-12-25 -1907-05-05 12:34:56.1877,1907-12-25 -1907-05-05 12:34:56.1878,1907-12-25 -1907-05-05 12:34:56.1879,1907-12-25 -1907-05-05 12:34:56.188,1907-12-25 -1907-05-05 12:34:56.1881,1907-12-25 -1907-05-05 12:34:56.1882,1907-12-25 -1907-05-05 12:34:56.1883,1907-12-25 -1907-05-05 12:34:56.1884,1907-12-25 -1907-05-05 12:34:56.1885,1907-12-25 -1907-05-05 12:34:56.1886,1907-12-25 -1907-05-05 12:34:56.1887,1907-12-25 -1907-05-05 12:34:56.1888,1907-12-25 -1907-05-05 12:34:56.1889,1907-12-25 -1907-05-05 12:34:56.189,1907-12-25 -1907-05-05 12:34:56.1891,1907-12-25 -1907-05-05 12:34:56.1892,1907-12-25 -1907-05-05 12:34:56.1893,1907-12-25 -1907-05-05 12:34:56.1894,1907-12-25 -1907-05-05 12:34:56.1895,1907-12-25 -1907-05-05 12:34:56.1896,1907-12-25 -1907-05-05 12:34:56.1897,1907-12-25 -1907-05-05 12:34:56.1898,1907-12-25 -1907-05-05 12:34:56.1899,1907-12-25 -1907-05-05 12:34:56.19,1907-12-25 -1907-05-05 12:34:56.1901,1907-12-25 -1907-05-05 12:34:56.1902,1907-12-25 -1907-05-05 12:34:56.1903,1907-12-25 -1907-05-05 12:34:56.1904,1907-12-25 -1907-05-05 12:34:56.1905,1907-12-25 -1907-05-05 12:34:56.1906,1907-12-25 -1907-05-05 12:34:56.1907,1907-12-25 -1907-05-05 12:34:56.1908,1907-12-25 -1907-05-05 12:34:56.1909,1907-12-25 -1907-05-05 12:34:56.191,1907-12-25 -1907-05-05 12:34:56.1911,1907-12-25 -1907-05-05 12:34:56.1912,1907-12-25 -1907-05-05 12:34:56.1913,1907-12-25 -1907-05-05 12:34:56.1914,1907-12-25 -1907-05-05 12:34:56.1915,1907-12-25 -1907-05-05 12:34:56.1916,1907-12-25 -1907-05-05 12:34:56.1917,1907-12-25 -1907-05-05 12:34:56.1918,1907-12-25 -1907-05-05 12:34:56.1919,1907-12-25 -1907-05-05 12:34:56.192,1907-12-25 -1907-05-05 12:34:56.1921,1907-12-25 -1907-05-05 12:34:56.1922,1907-12-25 -1907-05-05 12:34:56.1923,1907-12-25 -1907-05-05 12:34:56.1924,1907-12-25 -1907-05-05 12:34:56.1925,1907-12-25 -1907-05-05 12:34:56.1926,1907-12-25 -1907-05-05 12:34:56.1927,1907-12-25 -1907-05-05 12:34:56.1928,1907-12-25 -1907-05-05 12:34:56.1929,1907-12-25 -1907-05-05 12:34:56.193,1907-12-25 -1907-05-05 12:34:56.1931,1907-12-25 -1907-05-05 12:34:56.1932,1907-12-25 -1907-05-05 12:34:56.1933,1907-12-25 -1907-05-05 12:34:56.1934,1907-12-25 -1907-05-05 12:34:56.1935,1907-12-25 -1907-05-05 12:34:56.1936,1907-12-25 -1907-05-05 12:34:56.1937,1907-12-25 -1907-05-05 12:34:56.1938,1907-12-25 -1907-05-05 12:34:56.1939,1907-12-25 -1907-05-05 12:34:56.194,1907-12-25 -1907-05-05 12:34:56.1941,1907-12-25 -1907-05-05 12:34:56.1942,1907-12-25 -1907-05-05 12:34:56.1943,1907-12-25 -1907-05-05 12:34:56.1944,1907-12-25 -1907-05-05 12:34:56.1945,1907-12-25 -1907-05-05 12:34:56.1946,1907-12-25 -1907-05-05 12:34:56.1947,1907-12-25 -1907-05-05 12:34:56.1948,1907-12-25 -1907-05-05 12:34:56.1949,1907-12-25 -1907-05-05 12:34:56.195,1907-12-25 -1907-05-05 12:34:56.1951,1907-12-25 -1907-05-05 12:34:56.1952,1907-12-25 -1907-05-05 12:34:56.1953,1907-12-25 -1907-05-05 12:34:56.1954,1907-12-25 -1907-05-05 12:34:56.1955,1907-12-25 -1907-05-05 12:34:56.1956,1907-12-25 -1907-05-05 12:34:56.1957,1907-12-25 -1907-05-05 12:34:56.1958,1907-12-25 -1907-05-05 12:34:56.1959,1907-12-25 -1907-05-05 12:34:56.196,1907-12-25 -1907-05-05 12:34:56.1961,1907-12-25 -1907-05-05 12:34:56.1962,1907-12-25 -1907-05-05 12:34:56.1963,1907-12-25 -1907-05-05 12:34:56.1964,1907-12-25 -1907-05-05 12:34:56.1965,1907-12-25 -1907-05-05 12:34:56.1966,1907-12-25 -1907-05-05 12:34:56.1967,1907-12-25 -1907-05-05 12:34:56.1968,1907-12-25 -1907-05-05 12:34:56.1969,1907-12-25 -1907-05-05 12:34:56.197,1907-12-25 -1907-05-05 12:34:56.1971,1907-12-25 -1907-05-05 12:34:56.1972,1907-12-25 -1907-05-05 12:34:56.1973,1907-12-25 -1907-05-05 12:34:56.1974,1907-12-25 -1907-05-05 12:34:56.1975,1907-12-25 -1907-05-05 12:34:56.1976,1907-12-25 -1907-05-05 12:34:56.1977,1907-12-25 -1907-05-05 12:34:56.1978,1907-12-25 -1907-05-05 12:34:56.1979,1907-12-25 -1907-05-05 12:34:56.198,1907-12-25 -1907-05-05 12:34:56.1981,1907-12-25 -1907-05-05 12:34:56.1982,1907-12-25 -1907-05-05 12:34:56.1983,1907-12-25 -1907-05-05 12:34:56.1984,1907-12-25 -1907-05-05 12:34:56.1985,1907-12-25 -1907-05-05 12:34:56.1986,1907-12-25 -1907-05-05 12:34:56.1987,1907-12-25 -1907-05-05 12:34:56.1988,1907-12-25 -1907-05-05 12:34:56.1989,1907-12-25 -1907-05-05 12:34:56.199,1907-12-25 -1907-05-05 12:34:56.1991,1907-12-25 -1907-05-05 12:34:56.1992,1907-12-25 -1907-05-05 12:34:56.1993,1907-12-25 -1907-05-05 12:34:56.1994,1907-12-25 -1907-05-05 12:34:56.1995,1907-12-25 -1907-05-05 12:34:56.1996,1907-12-25 -1907-05-05 12:34:56.1997,1907-12-25 -1907-05-05 12:34:56.1998,1907-12-25 -1907-05-05 12:34:56.1999,1907-12-25 -1908-05-05 12:34:56.1,1908-12-25 -1908-05-05 12:34:56.1001,1908-12-25 -1908-05-05 12:34:56.1002,1908-12-25 -1908-05-05 12:34:56.1003,1908-12-25 -1908-05-05 12:34:56.1004,1908-12-25 -1908-05-05 12:34:56.1005,1908-12-25 -1908-05-05 12:34:56.1006,1908-12-25 -1908-05-05 12:34:56.1007,1908-12-25 -1908-05-05 12:34:56.1008,1908-12-25 -1908-05-05 12:34:56.1009,1908-12-25 -1908-05-05 12:34:56.101,1908-12-25 -1908-05-05 12:34:56.1011,1908-12-25 -1908-05-05 12:34:56.1012,1908-12-25 -1908-05-05 12:34:56.1013,1908-12-25 -1908-05-05 12:34:56.1014,1908-12-25 -1908-05-05 12:34:56.1015,1908-12-25 -1908-05-05 12:34:56.1016,1908-12-25 -1908-05-05 12:34:56.1017,1908-12-25 -1908-05-05 12:34:56.1018,1908-12-25 -1908-05-05 12:34:56.1019,1908-12-25 -1908-05-05 12:34:56.102,1908-12-25 -1908-05-05 12:34:56.1021,1908-12-25 -1908-05-05 12:34:56.1022,1908-12-25 -1908-05-05 12:34:56.1023,1908-12-25 -1908-05-05 12:34:56.1024,1908-12-25 -1908-05-05 12:34:56.1025,1908-12-25 -1908-05-05 12:34:56.1026,1908-12-25 -1908-05-05 12:34:56.1027,1908-12-25 -1908-05-05 12:34:56.1028,1908-12-25 -1908-05-05 12:34:56.1029,1908-12-25 -1908-05-05 12:34:56.103,1908-12-25 -1908-05-05 12:34:56.1031,1908-12-25 -1908-05-05 12:34:56.1032,1908-12-25 -1908-05-05 12:34:56.1033,1908-12-25 -1908-05-05 12:34:56.1034,1908-12-25 -1908-05-05 12:34:56.1035,1908-12-25 -1908-05-05 12:34:56.1036,1908-12-25 -1908-05-05 12:34:56.1037,1908-12-25 -1908-05-05 12:34:56.1038,1908-12-25 -1908-05-05 12:34:56.1039,1908-12-25 -1908-05-05 12:34:56.104,1908-12-25 -1908-05-05 12:34:56.1041,1908-12-25 -1908-05-05 12:34:56.1042,1908-12-25 -1908-05-05 12:34:56.1043,1908-12-25 -1908-05-05 12:34:56.1044,1908-12-25 -1908-05-05 12:34:56.1045,1908-12-25 -1908-05-05 12:34:56.1046,1908-12-25 -1908-05-05 12:34:56.1047,1908-12-25 -1908-05-05 12:34:56.1048,1908-12-25 -1908-05-05 12:34:56.1049,1908-12-25 -1908-05-05 12:34:56.105,1908-12-25 -1908-05-05 12:34:56.1051,1908-12-25 -1908-05-05 12:34:56.1052,1908-12-25 -1908-05-05 12:34:56.1053,1908-12-25 -1908-05-05 12:34:56.1054,1908-12-25 -1908-05-05 12:34:56.1055,1908-12-25 -1908-05-05 12:34:56.1056,1908-12-25 -1908-05-05 12:34:56.1057,1908-12-25 -1908-05-05 12:34:56.1058,1908-12-25 -1908-05-05 12:34:56.1059,1908-12-25 -1908-05-05 12:34:56.106,1908-12-25 -1908-05-05 12:34:56.1061,1908-12-25 -1908-05-05 12:34:56.1062,1908-12-25 -1908-05-05 12:34:56.1063,1908-12-25 -1908-05-05 12:34:56.1064,1908-12-25 -1908-05-05 12:34:56.1065,1908-12-25 -1908-05-05 12:34:56.1066,1908-12-25 -1908-05-05 12:34:56.1067,1908-12-25 -1908-05-05 12:34:56.1068,1908-12-25 -1908-05-05 12:34:56.1069,1908-12-25 -1908-05-05 12:34:56.107,1908-12-25 -1908-05-05 12:34:56.1071,1908-12-25 -1908-05-05 12:34:56.1072,1908-12-25 -1908-05-05 12:34:56.1073,1908-12-25 -1908-05-05 12:34:56.1074,1908-12-25 -1908-05-05 12:34:56.1075,1908-12-25 -1908-05-05 12:34:56.1076,1908-12-25 -1908-05-05 12:34:56.1077,1908-12-25 -1908-05-05 12:34:56.1078,1908-12-25 -1908-05-05 12:34:56.1079,1908-12-25 -1908-05-05 12:34:56.108,1908-12-25 -1908-05-05 12:34:56.1081,1908-12-25 -1908-05-05 12:34:56.1082,1908-12-25 -1908-05-05 12:34:56.1083,1908-12-25 -1908-05-05 12:34:56.1084,1908-12-25 -1908-05-05 12:34:56.1085,1908-12-25 -1908-05-05 12:34:56.1086,1908-12-25 -1908-05-05 12:34:56.1087,1908-12-25 -1908-05-05 12:34:56.1088,1908-12-25 -1908-05-05 12:34:56.1089,1908-12-25 -1908-05-05 12:34:56.109,1908-12-25 -1908-05-05 12:34:56.1091,1908-12-25 -1908-05-05 12:34:56.1092,1908-12-25 -1908-05-05 12:34:56.1093,1908-12-25 -1908-05-05 12:34:56.1094,1908-12-25 -1908-05-05 12:34:56.1095,1908-12-25 -1908-05-05 12:34:56.1096,1908-12-25 -1908-05-05 12:34:56.1097,1908-12-25 -1908-05-05 12:34:56.1098,1908-12-25 -1908-05-05 12:34:56.1099,1908-12-25 -1908-05-05 12:34:56.11,1908-12-25 -1908-05-05 12:34:56.1101,1908-12-25 -1908-05-05 12:34:56.1102,1908-12-25 -1908-05-05 12:34:56.1103,1908-12-25 -1908-05-05 12:34:56.1104,1908-12-25 -1908-05-05 12:34:56.1105,1908-12-25 -1908-05-05 12:34:56.1106,1908-12-25 -1908-05-05 12:34:56.1107,1908-12-25 -1908-05-05 12:34:56.1108,1908-12-25 -1908-05-05 12:34:56.1109,1908-12-25 -1908-05-05 12:34:56.111,1908-12-25 -1908-05-05 12:34:56.1111,1908-12-25 -1908-05-05 12:34:56.1112,1908-12-25 -1908-05-05 12:34:56.1113,1908-12-25 -1908-05-05 12:34:56.1114,1908-12-25 -1908-05-05 12:34:56.1115,1908-12-25 -1908-05-05 12:34:56.1116,1908-12-25 -1908-05-05 12:34:56.1117,1908-12-25 -1908-05-05 12:34:56.1118,1908-12-25 -1908-05-05 12:34:56.1119,1908-12-25 -1908-05-05 12:34:56.112,1908-12-25 -1908-05-05 12:34:56.1121,1908-12-25 -1908-05-05 12:34:56.1122,1908-12-25 -1908-05-05 12:34:56.1123,1908-12-25 -1908-05-05 12:34:56.1124,1908-12-25 -1908-05-05 12:34:56.1125,1908-12-25 -1908-05-05 12:34:56.1126,1908-12-25 -1908-05-05 12:34:56.1127,1908-12-25 -1908-05-05 12:34:56.1128,1908-12-25 -1908-05-05 12:34:56.1129,1908-12-25 -1908-05-05 12:34:56.113,1908-12-25 -1908-05-05 12:34:56.1131,1908-12-25 -1908-05-05 12:34:56.1132,1908-12-25 -1908-05-05 12:34:56.1133,1908-12-25 -1908-05-05 12:34:56.1134,1908-12-25 -1908-05-05 12:34:56.1135,1908-12-25 -1908-05-05 12:34:56.1136,1908-12-25 -1908-05-05 12:34:56.1137,1908-12-25 -1908-05-05 12:34:56.1138,1908-12-25 -1908-05-05 12:34:56.1139,1908-12-25 -1908-05-05 12:34:56.114,1908-12-25 -1908-05-05 12:34:56.1141,1908-12-25 -1908-05-05 12:34:56.1142,1908-12-25 -1908-05-05 12:34:56.1143,1908-12-25 -1908-05-05 12:34:56.1144,1908-12-25 -1908-05-05 12:34:56.1145,1908-12-25 -1908-05-05 12:34:56.1146,1908-12-25 -1908-05-05 12:34:56.1147,1908-12-25 -1908-05-05 12:34:56.1148,1908-12-25 -1908-05-05 12:34:56.1149,1908-12-25 -1908-05-05 12:34:56.115,1908-12-25 -1908-05-05 12:34:56.1151,1908-12-25 -1908-05-05 12:34:56.1152,1908-12-25 -1908-05-05 12:34:56.1153,1908-12-25 -1908-05-05 12:34:56.1154,1908-12-25 -1908-05-05 12:34:56.1155,1908-12-25 -1908-05-05 12:34:56.1156,1908-12-25 -1908-05-05 12:34:56.1157,1908-12-25 -1908-05-05 12:34:56.1158,1908-12-25 -1908-05-05 12:34:56.1159,1908-12-25 -1908-05-05 12:34:56.116,1908-12-25 -1908-05-05 12:34:56.1161,1908-12-25 -1908-05-05 12:34:56.1162,1908-12-25 -1908-05-05 12:34:56.1163,1908-12-25 -1908-05-05 12:34:56.1164,1908-12-25 -1908-05-05 12:34:56.1165,1908-12-25 -1908-05-05 12:34:56.1166,1908-12-25 -1908-05-05 12:34:56.1167,1908-12-25 -1908-05-05 12:34:56.1168,1908-12-25 -1908-05-05 12:34:56.1169,1908-12-25 -1908-05-05 12:34:56.117,1908-12-25 -1908-05-05 12:34:56.1171,1908-12-25 -1908-05-05 12:34:56.1172,1908-12-25 -1908-05-05 12:34:56.1173,1908-12-25 -1908-05-05 12:34:56.1174,1908-12-25 -1908-05-05 12:34:56.1175,1908-12-25 -1908-05-05 12:34:56.1176,1908-12-25 -1908-05-05 12:34:56.1177,1908-12-25 -1908-05-05 12:34:56.1178,1908-12-25 -1908-05-05 12:34:56.1179,1908-12-25 -1908-05-05 12:34:56.118,1908-12-25 -1908-05-05 12:34:56.1181,1908-12-25 -1908-05-05 12:34:56.1182,1908-12-25 -1908-05-05 12:34:56.1183,1908-12-25 -1908-05-05 12:34:56.1184,1908-12-25 -1908-05-05 12:34:56.1185,1908-12-25 -1908-05-05 12:34:56.1186,1908-12-25 -1908-05-05 12:34:56.1187,1908-12-25 -1908-05-05 12:34:56.1188,1908-12-25 -1908-05-05 12:34:56.1189,1908-12-25 -1908-05-05 12:34:56.119,1908-12-25 -1908-05-05 12:34:56.1191,1908-12-25 -1908-05-05 12:34:56.1192,1908-12-25 -1908-05-05 12:34:56.1193,1908-12-25 -1908-05-05 12:34:56.1194,1908-12-25 -1908-05-05 12:34:56.1195,1908-12-25 -1908-05-05 12:34:56.1196,1908-12-25 -1908-05-05 12:34:56.1197,1908-12-25 -1908-05-05 12:34:56.1198,1908-12-25 -1908-05-05 12:34:56.1199,1908-12-25 -1908-05-05 12:34:56.12,1908-12-25 -1908-05-05 12:34:56.1201,1908-12-25 -1908-05-05 12:34:56.1202,1908-12-25 -1908-05-05 12:34:56.1203,1908-12-25 -1908-05-05 12:34:56.1204,1908-12-25 -1908-05-05 12:34:56.1205,1908-12-25 -1908-05-05 12:34:56.1206,1908-12-25 -1908-05-05 12:34:56.1207,1908-12-25 -1908-05-05 12:34:56.1208,1908-12-25 -1908-05-05 12:34:56.1209,1908-12-25 -1908-05-05 12:34:56.121,1908-12-25 -1908-05-05 12:34:56.1211,1908-12-25 -1908-05-05 12:34:56.1212,1908-12-25 -1908-05-05 12:34:56.1213,1908-12-25 -1908-05-05 12:34:56.1214,1908-12-25 -1908-05-05 12:34:56.1215,1908-12-25 -1908-05-05 12:34:56.1216,1908-12-25 -1908-05-05 12:34:56.1217,1908-12-25 -1908-05-05 12:34:56.1218,1908-12-25 -1908-05-05 12:34:56.1219,1908-12-25 -1908-05-05 12:34:56.122,1908-12-25 -1908-05-05 12:34:56.1221,1908-12-25 -1908-05-05 12:34:56.1222,1908-12-25 -1908-05-05 12:34:56.1223,1908-12-25 -1908-05-05 12:34:56.1224,1908-12-25 -1908-05-05 12:34:56.1225,1908-12-25 -1908-05-05 12:34:56.1226,1908-12-25 -1908-05-05 12:34:56.1227,1908-12-25 -1908-05-05 12:34:56.1228,1908-12-25 -1908-05-05 12:34:56.1229,1908-12-25 -1908-05-05 12:34:56.123,1908-12-25 -1908-05-05 12:34:56.1231,1908-12-25 -1908-05-05 12:34:56.1232,1908-12-25 -1908-05-05 12:34:56.1233,1908-12-25 -1908-05-05 12:34:56.1234,1908-12-25 -1908-05-05 12:34:56.1235,1908-12-25 -1908-05-05 12:34:56.1236,1908-12-25 -1908-05-05 12:34:56.1237,1908-12-25 -1908-05-05 12:34:56.1238,1908-12-25 -1908-05-05 12:34:56.1239,1908-12-25 -1908-05-05 12:34:56.124,1908-12-25 -1908-05-05 12:34:56.1241,1908-12-25 -1908-05-05 12:34:56.1242,1908-12-25 -1908-05-05 12:34:56.1243,1908-12-25 -1908-05-05 12:34:56.1244,1908-12-25 -1908-05-05 12:34:56.1245,1908-12-25 -1908-05-05 12:34:56.1246,1908-12-25 -1908-05-05 12:34:56.1247,1908-12-25 -1908-05-05 12:34:56.1248,1908-12-25 -1908-05-05 12:34:56.1249,1908-12-25 -1908-05-05 12:34:56.125,1908-12-25 -1908-05-05 12:34:56.1251,1908-12-25 -1908-05-05 12:34:56.1252,1908-12-25 -1908-05-05 12:34:56.1253,1908-12-25 -1908-05-05 12:34:56.1254,1908-12-25 -1908-05-05 12:34:56.1255,1908-12-25 -1908-05-05 12:34:56.1256,1908-12-25 -1908-05-05 12:34:56.1257,1908-12-25 -1908-05-05 12:34:56.1258,1908-12-25 -1908-05-05 12:34:56.1259,1908-12-25 -1908-05-05 12:34:56.126,1908-12-25 -1908-05-05 12:34:56.1261,1908-12-25 -1908-05-05 12:34:56.1262,1908-12-25 -1908-05-05 12:34:56.1263,1908-12-25 -1908-05-05 12:34:56.1264,1908-12-25 -1908-05-05 12:34:56.1265,1908-12-25 -1908-05-05 12:34:56.1266,1908-12-25 -1908-05-05 12:34:56.1267,1908-12-25 -1908-05-05 12:34:56.1268,1908-12-25 -1908-05-05 12:34:56.1269,1908-12-25 -1908-05-05 12:34:56.127,1908-12-25 -1908-05-05 12:34:56.1271,1908-12-25 -1908-05-05 12:34:56.1272,1908-12-25 -1908-05-05 12:34:56.1273,1908-12-25 -1908-05-05 12:34:56.1274,1908-12-25 -1908-05-05 12:34:56.1275,1908-12-25 -1908-05-05 12:34:56.1276,1908-12-25 -1908-05-05 12:34:56.1277,1908-12-25 -1908-05-05 12:34:56.1278,1908-12-25 -1908-05-05 12:34:56.1279,1908-12-25 -1908-05-05 12:34:56.128,1908-12-25 -1908-05-05 12:34:56.1281,1908-12-25 -1908-05-05 12:34:56.1282,1908-12-25 -1908-05-05 12:34:56.1283,1908-12-25 -1908-05-05 12:34:56.1284,1908-12-25 -1908-05-05 12:34:56.1285,1908-12-25 -1908-05-05 12:34:56.1286,1908-12-25 -1908-05-05 12:34:56.1287,1908-12-25 -1908-05-05 12:34:56.1288,1908-12-25 -1908-05-05 12:34:56.1289,1908-12-25 -1908-05-05 12:34:56.129,1908-12-25 -1908-05-05 12:34:56.1291,1908-12-25 -1908-05-05 12:34:56.1292,1908-12-25 -1908-05-05 12:34:56.1293,1908-12-25 -1908-05-05 12:34:56.1294,1908-12-25 -1908-05-05 12:34:56.1295,1908-12-25 -1908-05-05 12:34:56.1296,1908-12-25 -1908-05-05 12:34:56.1297,1908-12-25 -1908-05-05 12:34:56.1298,1908-12-25 -1908-05-05 12:34:56.1299,1908-12-25 -1908-05-05 12:34:56.13,1908-12-25 -1908-05-05 12:34:56.1301,1908-12-25 -1908-05-05 12:34:56.1302,1908-12-25 -1908-05-05 12:34:56.1303,1908-12-25 -1908-05-05 12:34:56.1304,1908-12-25 -1908-05-05 12:34:56.1305,1908-12-25 -1908-05-05 12:34:56.1306,1908-12-25 -1908-05-05 12:34:56.1307,1908-12-25 -1908-05-05 12:34:56.1308,1908-12-25 -1908-05-05 12:34:56.1309,1908-12-25 -1908-05-05 12:34:56.131,1908-12-25 -1908-05-05 12:34:56.1311,1908-12-25 -1908-05-05 12:34:56.1312,1908-12-25 -1908-05-05 12:34:56.1313,1908-12-25 -1908-05-05 12:34:56.1314,1908-12-25 -1908-05-05 12:34:56.1315,1908-12-25 -1908-05-05 12:34:56.1316,1908-12-25 -1908-05-05 12:34:56.1317,1908-12-25 -1908-05-05 12:34:56.1318,1908-12-25 -1908-05-05 12:34:56.1319,1908-12-25 -1908-05-05 12:34:56.132,1908-12-25 -1908-05-05 12:34:56.1321,1908-12-25 -1908-05-05 12:34:56.1322,1908-12-25 -1908-05-05 12:34:56.1323,1908-12-25 -1908-05-05 12:34:56.1324,1908-12-25 -1908-05-05 12:34:56.1325,1908-12-25 -1908-05-05 12:34:56.1326,1908-12-25 -1908-05-05 12:34:56.1327,1908-12-25 -1908-05-05 12:34:56.1328,1908-12-25 -1908-05-05 12:34:56.1329,1908-12-25 -1908-05-05 12:34:56.133,1908-12-25 -1908-05-05 12:34:56.1331,1908-12-25 -1908-05-05 12:34:56.1332,1908-12-25 -1908-05-05 12:34:56.1333,1908-12-25 -1908-05-05 12:34:56.1334,1908-12-25 -1908-05-05 12:34:56.1335,1908-12-25 -1908-05-05 12:34:56.1336,1908-12-25 -1908-05-05 12:34:56.1337,1908-12-25 -1908-05-05 12:34:56.1338,1908-12-25 -1908-05-05 12:34:56.1339,1908-12-25 -1908-05-05 12:34:56.134,1908-12-25 -1908-05-05 12:34:56.1341,1908-12-25 -1908-05-05 12:34:56.1342,1908-12-25 -1908-05-05 12:34:56.1343,1908-12-25 -1908-05-05 12:34:56.1344,1908-12-25 -1908-05-05 12:34:56.1345,1908-12-25 -1908-05-05 12:34:56.1346,1908-12-25 -1908-05-05 12:34:56.1347,1908-12-25 -1908-05-05 12:34:56.1348,1908-12-25 -1908-05-05 12:34:56.1349,1908-12-25 -1908-05-05 12:34:56.135,1908-12-25 -1908-05-05 12:34:56.1351,1908-12-25 -1908-05-05 12:34:56.1352,1908-12-25 -1908-05-05 12:34:56.1353,1908-12-25 -1908-05-05 12:34:56.1354,1908-12-25 -1908-05-05 12:34:56.1355,1908-12-25 -1908-05-05 12:34:56.1356,1908-12-25 -1908-05-05 12:34:56.1357,1908-12-25 -1908-05-05 12:34:56.1358,1908-12-25 -1908-05-05 12:34:56.1359,1908-12-25 -1908-05-05 12:34:56.136,1908-12-25 -1908-05-05 12:34:56.1361,1908-12-25 -1908-05-05 12:34:56.1362,1908-12-25 -1908-05-05 12:34:56.1363,1908-12-25 -1908-05-05 12:34:56.1364,1908-12-25 -1908-05-05 12:34:56.1365,1908-12-25 -1908-05-05 12:34:56.1366,1908-12-25 -1908-05-05 12:34:56.1367,1908-12-25 -1908-05-05 12:34:56.1368,1908-12-25 -1908-05-05 12:34:56.1369,1908-12-25 -1908-05-05 12:34:56.137,1908-12-25 -1908-05-05 12:34:56.1371,1908-12-25 -1908-05-05 12:34:56.1372,1908-12-25 -1908-05-05 12:34:56.1373,1908-12-25 -1908-05-05 12:34:56.1374,1908-12-25 -1908-05-05 12:34:56.1375,1908-12-25 -1908-05-05 12:34:56.1376,1908-12-25 -1908-05-05 12:34:56.1377,1908-12-25 -1908-05-05 12:34:56.1378,1908-12-25 -1908-05-05 12:34:56.1379,1908-12-25 -1908-05-05 12:34:56.138,1908-12-25 -1908-05-05 12:34:56.1381,1908-12-25 -1908-05-05 12:34:56.1382,1908-12-25 -1908-05-05 12:34:56.1383,1908-12-25 -1908-05-05 12:34:56.1384,1908-12-25 -1908-05-05 12:34:56.1385,1908-12-25 -1908-05-05 12:34:56.1386,1908-12-25 -1908-05-05 12:34:56.1387,1908-12-25 -1908-05-05 12:34:56.1388,1908-12-25 -1908-05-05 12:34:56.1389,1908-12-25 -1908-05-05 12:34:56.139,1908-12-25 -1908-05-05 12:34:56.1391,1908-12-25 -1908-05-05 12:34:56.1392,1908-12-25 -1908-05-05 12:34:56.1393,1908-12-25 -1908-05-05 12:34:56.1394,1908-12-25 -1908-05-05 12:34:56.1395,1908-12-25 -1908-05-05 12:34:56.1396,1908-12-25 -1908-05-05 12:34:56.1397,1908-12-25 -1908-05-05 12:34:56.1398,1908-12-25 -1908-05-05 12:34:56.1399,1908-12-25 -1908-05-05 12:34:56.14,1908-12-25 -1908-05-05 12:34:56.1401,1908-12-25 -1908-05-05 12:34:56.1402,1908-12-25 -1908-05-05 12:34:56.1403,1908-12-25 -1908-05-05 12:34:56.1404,1908-12-25 -1908-05-05 12:34:56.1405,1908-12-25 -1908-05-05 12:34:56.1406,1908-12-25 -1908-05-05 12:34:56.1407,1908-12-25 -1908-05-05 12:34:56.1408,1908-12-25 -1908-05-05 12:34:56.1409,1908-12-25 -1908-05-05 12:34:56.141,1908-12-25 -1908-05-05 12:34:56.1411,1908-12-25 -1908-05-05 12:34:56.1412,1908-12-25 -1908-05-05 12:34:56.1413,1908-12-25 -1908-05-05 12:34:56.1414,1908-12-25 -1908-05-05 12:34:56.1415,1908-12-25 -1908-05-05 12:34:56.1416,1908-12-25 -1908-05-05 12:34:56.1417,1908-12-25 -1908-05-05 12:34:56.1418,1908-12-25 -1908-05-05 12:34:56.1419,1908-12-25 -1908-05-05 12:34:56.142,1908-12-25 -1908-05-05 12:34:56.1421,1908-12-25 -1908-05-05 12:34:56.1422,1908-12-25 -1908-05-05 12:34:56.1423,1908-12-25 -1908-05-05 12:34:56.1424,1908-12-25 -1908-05-05 12:34:56.1425,1908-12-25 -1908-05-05 12:34:56.1426,1908-12-25 -1908-05-05 12:34:56.1427,1908-12-25 -1908-05-05 12:34:56.1428,1908-12-25 -1908-05-05 12:34:56.1429,1908-12-25 -1908-05-05 12:34:56.143,1908-12-25 -1908-05-05 12:34:56.1431,1908-12-25 -1908-05-05 12:34:56.1432,1908-12-25 -1908-05-05 12:34:56.1433,1908-12-25 -1908-05-05 12:34:56.1434,1908-12-25 -1908-05-05 12:34:56.1435,1908-12-25 -1908-05-05 12:34:56.1436,1908-12-25 -1908-05-05 12:34:56.1437,1908-12-25 -1908-05-05 12:34:56.1438,1908-12-25 -1908-05-05 12:34:56.1439,1908-12-25 -1908-05-05 12:34:56.144,1908-12-25 -1908-05-05 12:34:56.1441,1908-12-25 -1908-05-05 12:34:56.1442,1908-12-25 -1908-05-05 12:34:56.1443,1908-12-25 -1908-05-05 12:34:56.1444,1908-12-25 -1908-05-05 12:34:56.1445,1908-12-25 -1908-05-05 12:34:56.1446,1908-12-25 -1908-05-05 12:34:56.1447,1908-12-25 -1908-05-05 12:34:56.1448,1908-12-25 -1908-05-05 12:34:56.1449,1908-12-25 -1908-05-05 12:34:56.145,1908-12-25 -1908-05-05 12:34:56.1451,1908-12-25 -1908-05-05 12:34:56.1452,1908-12-25 -1908-05-05 12:34:56.1453,1908-12-25 -1908-05-05 12:34:56.1454,1908-12-25 -1908-05-05 12:34:56.1455,1908-12-25 -1908-05-05 12:34:56.1456,1908-12-25 -1908-05-05 12:34:56.1457,1908-12-25 -1908-05-05 12:34:56.1458,1908-12-25 -1908-05-05 12:34:56.1459,1908-12-25 -1908-05-05 12:34:56.146,1908-12-25 -1908-05-05 12:34:56.1461,1908-12-25 -1908-05-05 12:34:56.1462,1908-12-25 -1908-05-05 12:34:56.1463,1908-12-25 -1908-05-05 12:34:56.1464,1908-12-25 -1908-05-05 12:34:56.1465,1908-12-25 -1908-05-05 12:34:56.1466,1908-12-25 -1908-05-05 12:34:56.1467,1908-12-25 -1908-05-05 12:34:56.1468,1908-12-25 -1908-05-05 12:34:56.1469,1908-12-25 -1908-05-05 12:34:56.147,1908-12-25 -1908-05-05 12:34:56.1471,1908-12-25 -1908-05-05 12:34:56.1472,1908-12-25 -1908-05-05 12:34:56.1473,1908-12-25 -1908-05-05 12:34:56.1474,1908-12-25 -1908-05-05 12:34:56.1475,1908-12-25 -1908-05-05 12:34:56.1476,1908-12-25 -1908-05-05 12:34:56.1477,1908-12-25 -1908-05-05 12:34:56.1478,1908-12-25 -1908-05-05 12:34:56.1479,1908-12-25 -1908-05-05 12:34:56.148,1908-12-25 -1908-05-05 12:34:56.1481,1908-12-25 -1908-05-05 12:34:56.1482,1908-12-25 -1908-05-05 12:34:56.1483,1908-12-25 -1908-05-05 12:34:56.1484,1908-12-25 -1908-05-05 12:34:56.1485,1908-12-25 -1908-05-05 12:34:56.1486,1908-12-25 -1908-05-05 12:34:56.1487,1908-12-25 -1908-05-05 12:34:56.1488,1908-12-25 -1908-05-05 12:34:56.1489,1908-12-25 -1908-05-05 12:34:56.149,1908-12-25 -1908-05-05 12:34:56.1491,1908-12-25 -1908-05-05 12:34:56.1492,1908-12-25 -1908-05-05 12:34:56.1493,1908-12-25 -1908-05-05 12:34:56.1494,1908-12-25 -1908-05-05 12:34:56.1495,1908-12-25 -1908-05-05 12:34:56.1496,1908-12-25 -1908-05-05 12:34:56.1497,1908-12-25 -1908-05-05 12:34:56.1498,1908-12-25 -1908-05-05 12:34:56.1499,1908-12-25 -1908-05-05 12:34:56.15,1908-12-25 -1908-05-05 12:34:56.1501,1908-12-25 -1908-05-05 12:34:56.1502,1908-12-25 -1908-05-05 12:34:56.1503,1908-12-25 -1908-05-05 12:34:56.1504,1908-12-25 -1908-05-05 12:34:56.1505,1908-12-25 -1908-05-05 12:34:56.1506,1908-12-25 -1908-05-05 12:34:56.1507,1908-12-25 -1908-05-05 12:34:56.1508,1908-12-25 -1908-05-05 12:34:56.1509,1908-12-25 -1908-05-05 12:34:56.151,1908-12-25 -1908-05-05 12:34:56.1511,1908-12-25 -1908-05-05 12:34:56.1512,1908-12-25 -1908-05-05 12:34:56.1513,1908-12-25 -1908-05-05 12:34:56.1514,1908-12-25 -1908-05-05 12:34:56.1515,1908-12-25 -1908-05-05 12:34:56.1516,1908-12-25 -1908-05-05 12:34:56.1517,1908-12-25 -1908-05-05 12:34:56.1518,1908-12-25 -1908-05-05 12:34:56.1519,1908-12-25 -1908-05-05 12:34:56.152,1908-12-25 -1908-05-05 12:34:56.1521,1908-12-25 -1908-05-05 12:34:56.1522,1908-12-25 -1908-05-05 12:34:56.1523,1908-12-25 -1908-05-05 12:34:56.1524,1908-12-25 -1908-05-05 12:34:56.1525,1908-12-25 -1908-05-05 12:34:56.1526,1908-12-25 -1908-05-05 12:34:56.1527,1908-12-25 -1908-05-05 12:34:56.1528,1908-12-25 -1908-05-05 12:34:56.1529,1908-12-25 -1908-05-05 12:34:56.153,1908-12-25 -1908-05-05 12:34:56.1531,1908-12-25 -1908-05-05 12:34:56.1532,1908-12-25 -1908-05-05 12:34:56.1533,1908-12-25 -1908-05-05 12:34:56.1534,1908-12-25 -1908-05-05 12:34:56.1535,1908-12-25 -1908-05-05 12:34:56.1536,1908-12-25 -1908-05-05 12:34:56.1537,1908-12-25 -1908-05-05 12:34:56.1538,1908-12-25 -1908-05-05 12:34:56.1539,1908-12-25 -1908-05-05 12:34:56.154,1908-12-25 -1908-05-05 12:34:56.1541,1908-12-25 -1908-05-05 12:34:56.1542,1908-12-25 -1908-05-05 12:34:56.1543,1908-12-25 -1908-05-05 12:34:56.1544,1908-12-25 -1908-05-05 12:34:56.1545,1908-12-25 -1908-05-05 12:34:56.1546,1908-12-25 -1908-05-05 12:34:56.1547,1908-12-25 -1908-05-05 12:34:56.1548,1908-12-25 -1908-05-05 12:34:56.1549,1908-12-25 -1908-05-05 12:34:56.155,1908-12-25 -1908-05-05 12:34:56.1551,1908-12-25 -1908-05-05 12:34:56.1552,1908-12-25 -1908-05-05 12:34:56.1553,1908-12-25 -1908-05-05 12:34:56.1554,1908-12-25 -1908-05-05 12:34:56.1555,1908-12-25 -1908-05-05 12:34:56.1556,1908-12-25 -1908-05-05 12:34:56.1557,1908-12-25 -1908-05-05 12:34:56.1558,1908-12-25 -1908-05-05 12:34:56.1559,1908-12-25 -1908-05-05 12:34:56.156,1908-12-25 -1908-05-05 12:34:56.1561,1908-12-25 -1908-05-05 12:34:56.1562,1908-12-25 -1908-05-05 12:34:56.1563,1908-12-25 -1908-05-05 12:34:56.1564,1908-12-25 -1908-05-05 12:34:56.1565,1908-12-25 -1908-05-05 12:34:56.1566,1908-12-25 -1908-05-05 12:34:56.1567,1908-12-25 -1908-05-05 12:34:56.1568,1908-12-25 -1908-05-05 12:34:56.1569,1908-12-25 -1908-05-05 12:34:56.157,1908-12-25 -1908-05-05 12:34:56.1571,1908-12-25 -1908-05-05 12:34:56.1572,1908-12-25 -1908-05-05 12:34:56.1573,1908-12-25 -1908-05-05 12:34:56.1574,1908-12-25 -1908-05-05 12:34:56.1575,1908-12-25 -1908-05-05 12:34:56.1576,1908-12-25 -1908-05-05 12:34:56.1577,1908-12-25 -1908-05-05 12:34:56.1578,1908-12-25 -1908-05-05 12:34:56.1579,1908-12-25 -1908-05-05 12:34:56.158,1908-12-25 -1908-05-05 12:34:56.1581,1908-12-25 -1908-05-05 12:34:56.1582,1908-12-25 -1908-05-05 12:34:56.1583,1908-12-25 -1908-05-05 12:34:56.1584,1908-12-25 -1908-05-05 12:34:56.1585,1908-12-25 -1908-05-05 12:34:56.1586,1908-12-25 -1908-05-05 12:34:56.1587,1908-12-25 -1908-05-05 12:34:56.1588,1908-12-25 -1908-05-05 12:34:56.1589,1908-12-25 -1908-05-05 12:34:56.159,1908-12-25 -1908-05-05 12:34:56.1591,1908-12-25 -1908-05-05 12:34:56.1592,1908-12-25 -1908-05-05 12:34:56.1593,1908-12-25 -1908-05-05 12:34:56.1594,1908-12-25 -1908-05-05 12:34:56.1595,1908-12-25 -1908-05-05 12:34:56.1596,1908-12-25 -1908-05-05 12:34:56.1597,1908-12-25 -1908-05-05 12:34:56.1598,1908-12-25 -1908-05-05 12:34:56.1599,1908-12-25 -1908-05-05 12:34:56.16,1908-12-25 -1908-05-05 12:34:56.1601,1908-12-25 -1908-05-05 12:34:56.1602,1908-12-25 -1908-05-05 12:34:56.1603,1908-12-25 -1908-05-05 12:34:56.1604,1908-12-25 -1908-05-05 12:34:56.1605,1908-12-25 -1908-05-05 12:34:56.1606,1908-12-25 -1908-05-05 12:34:56.1607,1908-12-25 -1908-05-05 12:34:56.1608,1908-12-25 -1908-05-05 12:34:56.1609,1908-12-25 -1908-05-05 12:34:56.161,1908-12-25 -1908-05-05 12:34:56.1611,1908-12-25 -1908-05-05 12:34:56.1612,1908-12-25 -1908-05-05 12:34:56.1613,1908-12-25 -1908-05-05 12:34:56.1614,1908-12-25 -1908-05-05 12:34:56.1615,1908-12-25 -1908-05-05 12:34:56.1616,1908-12-25 -1908-05-05 12:34:56.1617,1908-12-25 -1908-05-05 12:34:56.1618,1908-12-25 -1908-05-05 12:34:56.1619,1908-12-25 -1908-05-05 12:34:56.162,1908-12-25 -1908-05-05 12:34:56.1621,1908-12-25 -1908-05-05 12:34:56.1622,1908-12-25 -1908-05-05 12:34:56.1623,1908-12-25 -1908-05-05 12:34:56.1624,1908-12-25 -1908-05-05 12:34:56.1625,1908-12-25 -1908-05-05 12:34:56.1626,1908-12-25 -1908-05-05 12:34:56.1627,1908-12-25 -1908-05-05 12:34:56.1628,1908-12-25 -1908-05-05 12:34:56.1629,1908-12-25 -1908-05-05 12:34:56.163,1908-12-25 -1908-05-05 12:34:56.1631,1908-12-25 -1908-05-05 12:34:56.1632,1908-12-25 -1908-05-05 12:34:56.1633,1908-12-25 -1908-05-05 12:34:56.1634,1908-12-25 -1908-05-05 12:34:56.1635,1908-12-25 -1908-05-05 12:34:56.1636,1908-12-25 -1908-05-05 12:34:56.1637,1908-12-25 -1908-05-05 12:34:56.1638,1908-12-25 -1908-05-05 12:34:56.1639,1908-12-25 -1908-05-05 12:34:56.164,1908-12-25 -1908-05-05 12:34:56.1641,1908-12-25 -1908-05-05 12:34:56.1642,1908-12-25 -1908-05-05 12:34:56.1643,1908-12-25 -1908-05-05 12:34:56.1644,1908-12-25 -1908-05-05 12:34:56.1645,1908-12-25 -1908-05-05 12:34:56.1646,1908-12-25 -1908-05-05 12:34:56.1647,1908-12-25 -1908-05-05 12:34:56.1648,1908-12-25 -1908-05-05 12:34:56.1649,1908-12-25 -1908-05-05 12:34:56.165,1908-12-25 -1908-05-05 12:34:56.1651,1908-12-25 -1908-05-05 12:34:56.1652,1908-12-25 -1908-05-05 12:34:56.1653,1908-12-25 -1908-05-05 12:34:56.1654,1908-12-25 -1908-05-05 12:34:56.1655,1908-12-25 -1908-05-05 12:34:56.1656,1908-12-25 -1908-05-05 12:34:56.1657,1908-12-25 -1908-05-05 12:34:56.1658,1908-12-25 -1908-05-05 12:34:56.1659,1908-12-25 -1908-05-05 12:34:56.166,1908-12-25 -1908-05-05 12:34:56.1661,1908-12-25 -1908-05-05 12:34:56.1662,1908-12-25 -1908-05-05 12:34:56.1663,1908-12-25 -1908-05-05 12:34:56.1664,1908-12-25 -1908-05-05 12:34:56.1665,1908-12-25 -1908-05-05 12:34:56.1666,1908-12-25 -1908-05-05 12:34:56.1667,1908-12-25 -1908-05-05 12:34:56.1668,1908-12-25 -1908-05-05 12:34:56.1669,1908-12-25 -1908-05-05 12:34:56.167,1908-12-25 -1908-05-05 12:34:56.1671,1908-12-25 -1908-05-05 12:34:56.1672,1908-12-25 -1908-05-05 12:34:56.1673,1908-12-25 -1908-05-05 12:34:56.1674,1908-12-25 -1908-05-05 12:34:56.1675,1908-12-25 -1908-05-05 12:34:56.1676,1908-12-25 -1908-05-05 12:34:56.1677,1908-12-25 -1908-05-05 12:34:56.1678,1908-12-25 -1908-05-05 12:34:56.1679,1908-12-25 -1908-05-05 12:34:56.168,1908-12-25 -1908-05-05 12:34:56.1681,1908-12-25 -1908-05-05 12:34:56.1682,1908-12-25 -1908-05-05 12:34:56.1683,1908-12-25 -1908-05-05 12:34:56.1684,1908-12-25 -1908-05-05 12:34:56.1685,1908-12-25 -1908-05-05 12:34:56.1686,1908-12-25 -1908-05-05 12:34:56.1687,1908-12-25 -1908-05-05 12:34:56.1688,1908-12-25 -1908-05-05 12:34:56.1689,1908-12-25 -1908-05-05 12:34:56.169,1908-12-25 -1908-05-05 12:34:56.1691,1908-12-25 -1908-05-05 12:34:56.1692,1908-12-25 -1908-05-05 12:34:56.1693,1908-12-25 -1908-05-05 12:34:56.1694,1908-12-25 -1908-05-05 12:34:56.1695,1908-12-25 -1908-05-05 12:34:56.1696,1908-12-25 -1908-05-05 12:34:56.1697,1908-12-25 -1908-05-05 12:34:56.1698,1908-12-25 -1908-05-05 12:34:56.1699,1908-12-25 -1908-05-05 12:34:56.17,1908-12-25 -1908-05-05 12:34:56.1701,1908-12-25 -1908-05-05 12:34:56.1702,1908-12-25 -1908-05-05 12:34:56.1703,1908-12-25 -1908-05-05 12:34:56.1704,1908-12-25 -1908-05-05 12:34:56.1705,1908-12-25 -1908-05-05 12:34:56.1706,1908-12-25 -1908-05-05 12:34:56.1707,1908-12-25 -1908-05-05 12:34:56.1708,1908-12-25 -1908-05-05 12:34:56.1709,1908-12-25 -1908-05-05 12:34:56.171,1908-12-25 -1908-05-05 12:34:56.1711,1908-12-25 -1908-05-05 12:34:56.1712,1908-12-25 -1908-05-05 12:34:56.1713,1908-12-25 -1908-05-05 12:34:56.1714,1908-12-25 -1908-05-05 12:34:56.1715,1908-12-25 -1908-05-05 12:34:56.1716,1908-12-25 -1908-05-05 12:34:56.1717,1908-12-25 -1908-05-05 12:34:56.1718,1908-12-25 -1908-05-05 12:34:56.1719,1908-12-25 -1908-05-05 12:34:56.172,1908-12-25 -1908-05-05 12:34:56.1721,1908-12-25 -1908-05-05 12:34:56.1722,1908-12-25 -1908-05-05 12:34:56.1723,1908-12-25 -1908-05-05 12:34:56.1724,1908-12-25 -1908-05-05 12:34:56.1725,1908-12-25 -1908-05-05 12:34:56.1726,1908-12-25 -1908-05-05 12:34:56.1727,1908-12-25 -1908-05-05 12:34:56.1728,1908-12-25 -1908-05-05 12:34:56.1729,1908-12-25 -1908-05-05 12:34:56.173,1908-12-25 -1908-05-05 12:34:56.1731,1908-12-25 -1908-05-05 12:34:56.1732,1908-12-25 -1908-05-05 12:34:56.1733,1908-12-25 -1908-05-05 12:34:56.1734,1908-12-25 -1908-05-05 12:34:56.1735,1908-12-25 -1908-05-05 12:34:56.1736,1908-12-25 -1908-05-05 12:34:56.1737,1908-12-25 -1908-05-05 12:34:56.1738,1908-12-25 -1908-05-05 12:34:56.1739,1908-12-25 -1908-05-05 12:34:56.174,1908-12-25 -1908-05-05 12:34:56.1741,1908-12-25 -1908-05-05 12:34:56.1742,1908-12-25 -1908-05-05 12:34:56.1743,1908-12-25 -1908-05-05 12:34:56.1744,1908-12-25 -1908-05-05 12:34:56.1745,1908-12-25 -1908-05-05 12:34:56.1746,1908-12-25 -1908-05-05 12:34:56.1747,1908-12-25 -1908-05-05 12:34:56.1748,1908-12-25 -1908-05-05 12:34:56.1749,1908-12-25 -1908-05-05 12:34:56.175,1908-12-25 -1908-05-05 12:34:56.1751,1908-12-25 -1908-05-05 12:34:56.1752,1908-12-25 -1908-05-05 12:34:56.1753,1908-12-25 -1908-05-05 12:34:56.1754,1908-12-25 -1908-05-05 12:34:56.1755,1908-12-25 -1908-05-05 12:34:56.1756,1908-12-25 -1908-05-05 12:34:56.1757,1908-12-25 -1908-05-05 12:34:56.1758,1908-12-25 -1908-05-05 12:34:56.1759,1908-12-25 -1908-05-05 12:34:56.176,1908-12-25 -1908-05-05 12:34:56.1761,1908-12-25 -1908-05-05 12:34:56.1762,1908-12-25 -1908-05-05 12:34:56.1763,1908-12-25 -1908-05-05 12:34:56.1764,1908-12-25 -1908-05-05 12:34:56.1765,1908-12-25 -1908-05-05 12:34:56.1766,1908-12-25 -1908-05-05 12:34:56.1767,1908-12-25 -1908-05-05 12:34:56.1768,1908-12-25 -1908-05-05 12:34:56.1769,1908-12-25 -1908-05-05 12:34:56.177,1908-12-25 -1908-05-05 12:34:56.1771,1908-12-25 -1908-05-05 12:34:56.1772,1908-12-25 -1908-05-05 12:34:56.1773,1908-12-25 -1908-05-05 12:34:56.1774,1908-12-25 -1908-05-05 12:34:56.1775,1908-12-25 -1908-05-05 12:34:56.1776,1908-12-25 -1908-05-05 12:34:56.1777,1908-12-25 -1908-05-05 12:34:56.1778,1908-12-25 -1908-05-05 12:34:56.1779,1908-12-25 -1908-05-05 12:34:56.178,1908-12-25 -1908-05-05 12:34:56.1781,1908-12-25 -1908-05-05 12:34:56.1782,1908-12-25 -1908-05-05 12:34:56.1783,1908-12-25 -1908-05-05 12:34:56.1784,1908-12-25 -1908-05-05 12:34:56.1785,1908-12-25 -1908-05-05 12:34:56.1786,1908-12-25 -1908-05-05 12:34:56.1787,1908-12-25 -1908-05-05 12:34:56.1788,1908-12-25 -1908-05-05 12:34:56.1789,1908-12-25 -1908-05-05 12:34:56.179,1908-12-25 -1908-05-05 12:34:56.1791,1908-12-25 -1908-05-05 12:34:56.1792,1908-12-25 -1908-05-05 12:34:56.1793,1908-12-25 -1908-05-05 12:34:56.1794,1908-12-25 -1908-05-05 12:34:56.1795,1908-12-25 -1908-05-05 12:34:56.1796,1908-12-25 -1908-05-05 12:34:56.1797,1908-12-25 -1908-05-05 12:34:56.1798,1908-12-25 -1908-05-05 12:34:56.1799,1908-12-25 -1908-05-05 12:34:56.18,1908-12-25 -1908-05-05 12:34:56.1801,1908-12-25 -1908-05-05 12:34:56.1802,1908-12-25 -1908-05-05 12:34:56.1803,1908-12-25 -1908-05-05 12:34:56.1804,1908-12-25 -1908-05-05 12:34:56.1805,1908-12-25 -1908-05-05 12:34:56.1806,1908-12-25 -1908-05-05 12:34:56.1807,1908-12-25 -1908-05-05 12:34:56.1808,1908-12-25 -1908-05-05 12:34:56.1809,1908-12-25 -1908-05-05 12:34:56.181,1908-12-25 -1908-05-05 12:34:56.1811,1908-12-25 -1908-05-05 12:34:56.1812,1908-12-25 -1908-05-05 12:34:56.1813,1908-12-25 -1908-05-05 12:34:56.1814,1908-12-25 -1908-05-05 12:34:56.1815,1908-12-25 -1908-05-05 12:34:56.1816,1908-12-25 -1908-05-05 12:34:56.1817,1908-12-25 -1908-05-05 12:34:56.1818,1908-12-25 -1908-05-05 12:34:56.1819,1908-12-25 -1908-05-05 12:34:56.182,1908-12-25 -1908-05-05 12:34:56.1821,1908-12-25 -1908-05-05 12:34:56.1822,1908-12-25 -1908-05-05 12:34:56.1823,1908-12-25 -1908-05-05 12:34:56.1824,1908-12-25 -1908-05-05 12:34:56.1825,1908-12-25 -1908-05-05 12:34:56.1826,1908-12-25 -1908-05-05 12:34:56.1827,1908-12-25 -1908-05-05 12:34:56.1828,1908-12-25 -1908-05-05 12:34:56.1829,1908-12-25 -1908-05-05 12:34:56.183,1908-12-25 -1908-05-05 12:34:56.1831,1908-12-25 -1908-05-05 12:34:56.1832,1908-12-25 -1908-05-05 12:34:56.1833,1908-12-25 -1908-05-05 12:34:56.1834,1908-12-25 -1908-05-05 12:34:56.1835,1908-12-25 -1908-05-05 12:34:56.1836,1908-12-25 -1908-05-05 12:34:56.1837,1908-12-25 -1908-05-05 12:34:56.1838,1908-12-25 -1908-05-05 12:34:56.1839,1908-12-25 -1908-05-05 12:34:56.184,1908-12-25 -1908-05-05 12:34:56.1841,1908-12-25 -1908-05-05 12:34:56.1842,1908-12-25 -1908-05-05 12:34:56.1843,1908-12-25 -1908-05-05 12:34:56.1844,1908-12-25 -1908-05-05 12:34:56.1845,1908-12-25 -1908-05-05 12:34:56.1846,1908-12-25 -1908-05-05 12:34:56.1847,1908-12-25 -1908-05-05 12:34:56.1848,1908-12-25 -1908-05-05 12:34:56.1849,1908-12-25 -1908-05-05 12:34:56.185,1908-12-25 -1908-05-05 12:34:56.1851,1908-12-25 -1908-05-05 12:34:56.1852,1908-12-25 -1908-05-05 12:34:56.1853,1908-12-25 -1908-05-05 12:34:56.1854,1908-12-25 -1908-05-05 12:34:56.1855,1908-12-25 -1908-05-05 12:34:56.1856,1908-12-25 -1908-05-05 12:34:56.1857,1908-12-25 -1908-05-05 12:34:56.1858,1908-12-25 -1908-05-05 12:34:56.1859,1908-12-25 -1908-05-05 12:34:56.186,1908-12-25 -1908-05-05 12:34:56.1861,1908-12-25 -1908-05-05 12:34:56.1862,1908-12-25 -1908-05-05 12:34:56.1863,1908-12-25 -1908-05-05 12:34:56.1864,1908-12-25 -1908-05-05 12:34:56.1865,1908-12-25 -1908-05-05 12:34:56.1866,1908-12-25 -1908-05-05 12:34:56.1867,1908-12-25 -1908-05-05 12:34:56.1868,1908-12-25 -1908-05-05 12:34:56.1869,1908-12-25 -1908-05-05 12:34:56.187,1908-12-25 -1908-05-05 12:34:56.1871,1908-12-25 -1908-05-05 12:34:56.1872,1908-12-25 -1908-05-05 12:34:56.1873,1908-12-25 -1908-05-05 12:34:56.1874,1908-12-25 -1908-05-05 12:34:56.1875,1908-12-25 -1908-05-05 12:34:56.1876,1908-12-25 -1908-05-05 12:34:56.1877,1908-12-25 -1908-05-05 12:34:56.1878,1908-12-25 -1908-05-05 12:34:56.1879,1908-12-25 -1908-05-05 12:34:56.188,1908-12-25 -1908-05-05 12:34:56.1881,1908-12-25 -1908-05-05 12:34:56.1882,1908-12-25 -1908-05-05 12:34:56.1883,1908-12-25 -1908-05-05 12:34:56.1884,1908-12-25 -1908-05-05 12:34:56.1885,1908-12-25 -1908-05-05 12:34:56.1886,1908-12-25 -1908-05-05 12:34:56.1887,1908-12-25 -1908-05-05 12:34:56.1888,1908-12-25 -1908-05-05 12:34:56.1889,1908-12-25 -1908-05-05 12:34:56.189,1908-12-25 -1908-05-05 12:34:56.1891,1908-12-25 -1908-05-05 12:34:56.1892,1908-12-25 -1908-05-05 12:34:56.1893,1908-12-25 -1908-05-05 12:34:56.1894,1908-12-25 -1908-05-05 12:34:56.1895,1908-12-25 -1908-05-05 12:34:56.1896,1908-12-25 -1908-05-05 12:34:56.1897,1908-12-25 -1908-05-05 12:34:56.1898,1908-12-25 -1908-05-05 12:34:56.1899,1908-12-25 -1908-05-05 12:34:56.19,1908-12-25 -1908-05-05 12:34:56.1901,1908-12-25 -1908-05-05 12:34:56.1902,1908-12-25 -1908-05-05 12:34:56.1903,1908-12-25 -1908-05-05 12:34:56.1904,1908-12-25 -1908-05-05 12:34:56.1905,1908-12-25 -1908-05-05 12:34:56.1906,1908-12-25 -1908-05-05 12:34:56.1907,1908-12-25 -1908-05-05 12:34:56.1908,1908-12-25 -1908-05-05 12:34:56.1909,1908-12-25 -1908-05-05 12:34:56.191,1908-12-25 -1908-05-05 12:34:56.1911,1908-12-25 -1908-05-05 12:34:56.1912,1908-12-25 -1908-05-05 12:34:56.1913,1908-12-25 -1908-05-05 12:34:56.1914,1908-12-25 -1908-05-05 12:34:56.1915,1908-12-25 -1908-05-05 12:34:56.1916,1908-12-25 -1908-05-05 12:34:56.1917,1908-12-25 -1908-05-05 12:34:56.1918,1908-12-25 -1908-05-05 12:34:56.1919,1908-12-25 -1908-05-05 12:34:56.192,1908-12-25 -1908-05-05 12:34:56.1921,1908-12-25 -1908-05-05 12:34:56.1922,1908-12-25 -1908-05-05 12:34:56.1923,1908-12-25 -1908-05-05 12:34:56.1924,1908-12-25 -1908-05-05 12:34:56.1925,1908-12-25 -1908-05-05 12:34:56.1926,1908-12-25 -1908-05-05 12:34:56.1927,1908-12-25 -1908-05-05 12:34:56.1928,1908-12-25 -1908-05-05 12:34:56.1929,1908-12-25 -1908-05-05 12:34:56.193,1908-12-25 -1908-05-05 12:34:56.1931,1908-12-25 -1908-05-05 12:34:56.1932,1908-12-25 -1908-05-05 12:34:56.1933,1908-12-25 -1908-05-05 12:34:56.1934,1908-12-25 -1908-05-05 12:34:56.1935,1908-12-25 -1908-05-05 12:34:56.1936,1908-12-25 -1908-05-05 12:34:56.1937,1908-12-25 -1908-05-05 12:34:56.1938,1908-12-25 -1908-05-05 12:34:56.1939,1908-12-25 -1908-05-05 12:34:56.194,1908-12-25 -1908-05-05 12:34:56.1941,1908-12-25 -1908-05-05 12:34:56.1942,1908-12-25 -1908-05-05 12:34:56.1943,1908-12-25 -1908-05-05 12:34:56.1944,1908-12-25 -1908-05-05 12:34:56.1945,1908-12-25 -1908-05-05 12:34:56.1946,1908-12-25 -1908-05-05 12:34:56.1947,1908-12-25 -1908-05-05 12:34:56.1948,1908-12-25 -1908-05-05 12:34:56.1949,1908-12-25 -1908-05-05 12:34:56.195,1908-12-25 -1908-05-05 12:34:56.1951,1908-12-25 -1908-05-05 12:34:56.1952,1908-12-25 -1908-05-05 12:34:56.1953,1908-12-25 -1908-05-05 12:34:56.1954,1908-12-25 -1908-05-05 12:34:56.1955,1908-12-25 -1908-05-05 12:34:56.1956,1908-12-25 -1908-05-05 12:34:56.1957,1908-12-25 -1908-05-05 12:34:56.1958,1908-12-25 -1908-05-05 12:34:56.1959,1908-12-25 -1908-05-05 12:34:56.196,1908-12-25 -1908-05-05 12:34:56.1961,1908-12-25 -1908-05-05 12:34:56.1962,1908-12-25 -1908-05-05 12:34:56.1963,1908-12-25 -1908-05-05 12:34:56.1964,1908-12-25 -1908-05-05 12:34:56.1965,1908-12-25 -1908-05-05 12:34:56.1966,1908-12-25 -1908-05-05 12:34:56.1967,1908-12-25 -1908-05-05 12:34:56.1968,1908-12-25 -1908-05-05 12:34:56.1969,1908-12-25 -1908-05-05 12:34:56.197,1908-12-25 -1908-05-05 12:34:56.1971,1908-12-25 -1908-05-05 12:34:56.1972,1908-12-25 -1908-05-05 12:34:56.1973,1908-12-25 -1908-05-05 12:34:56.1974,1908-12-25 -1908-05-05 12:34:56.1975,1908-12-25 -1908-05-05 12:34:56.1976,1908-12-25 -1908-05-05 12:34:56.1977,1908-12-25 -1908-05-05 12:34:56.1978,1908-12-25 -1908-05-05 12:34:56.1979,1908-12-25 -1908-05-05 12:34:56.198,1908-12-25 -1908-05-05 12:34:56.1981,1908-12-25 -1908-05-05 12:34:56.1982,1908-12-25 -1908-05-05 12:34:56.1983,1908-12-25 -1908-05-05 12:34:56.1984,1908-12-25 -1908-05-05 12:34:56.1985,1908-12-25 -1908-05-05 12:34:56.1986,1908-12-25 -1908-05-05 12:34:56.1987,1908-12-25 -1908-05-05 12:34:56.1988,1908-12-25 -1908-05-05 12:34:56.1989,1908-12-25 -1908-05-05 12:34:56.199,1908-12-25 -1908-05-05 12:34:56.1991,1908-12-25 -1908-05-05 12:34:56.1992,1908-12-25 -1908-05-05 12:34:56.1993,1908-12-25 -1908-05-05 12:34:56.1994,1908-12-25 -1908-05-05 12:34:56.1995,1908-12-25 -1908-05-05 12:34:56.1996,1908-12-25 -1908-05-05 12:34:56.1997,1908-12-25 -1908-05-05 12:34:56.1998,1908-12-25 -1908-05-05 12:34:56.1999,1908-12-25 -1909-05-05 12:34:56.1,1909-12-25 -1909-05-05 12:34:56.1001,1909-12-25 -1909-05-05 12:34:56.1002,1909-12-25 -1909-05-05 12:34:56.1003,1909-12-25 -1909-05-05 12:34:56.1004,1909-12-25 -1909-05-05 12:34:56.1005,1909-12-25 -1909-05-05 12:34:56.1006,1909-12-25 -1909-05-05 12:34:56.1007,1909-12-25 -1909-05-05 12:34:56.1008,1909-12-25 -1909-05-05 12:34:56.1009,1909-12-25 -1909-05-05 12:34:56.101,1909-12-25 -1909-05-05 12:34:56.1011,1909-12-25 -1909-05-05 12:34:56.1012,1909-12-25 -1909-05-05 12:34:56.1013,1909-12-25 -1909-05-05 12:34:56.1014,1909-12-25 -1909-05-05 12:34:56.1015,1909-12-25 -1909-05-05 12:34:56.1016,1909-12-25 -1909-05-05 12:34:56.1017,1909-12-25 -1909-05-05 12:34:56.1018,1909-12-25 -1909-05-05 12:34:56.1019,1909-12-25 -1909-05-05 12:34:56.102,1909-12-25 -1909-05-05 12:34:56.1021,1909-12-25 -1909-05-05 12:34:56.1022,1909-12-25 -1909-05-05 12:34:56.1023,1909-12-25 -1909-05-05 12:34:56.1024,1909-12-25 -1909-05-05 12:34:56.1025,1909-12-25 -1909-05-05 12:34:56.1026,1909-12-25 -1909-05-05 12:34:56.1027,1909-12-25 -1909-05-05 12:34:56.1028,1909-12-25 -1909-05-05 12:34:56.1029,1909-12-25 -1909-05-05 12:34:56.103,1909-12-25 -1909-05-05 12:34:56.1031,1909-12-25 -1909-05-05 12:34:56.1032,1909-12-25 -1909-05-05 12:34:56.1033,1909-12-25 -1909-05-05 12:34:56.1034,1909-12-25 -1909-05-05 12:34:56.1035,1909-12-25 -1909-05-05 12:34:56.1036,1909-12-25 -1909-05-05 12:34:56.1037,1909-12-25 -1909-05-05 12:34:56.1038,1909-12-25 -1909-05-05 12:34:56.1039,1909-12-25 -1909-05-05 12:34:56.104,1909-12-25 -1909-05-05 12:34:56.1041,1909-12-25 -1909-05-05 12:34:56.1042,1909-12-25 -1909-05-05 12:34:56.1043,1909-12-25 -1909-05-05 12:34:56.1044,1909-12-25 -1909-05-05 12:34:56.1045,1909-12-25 -1909-05-05 12:34:56.1046,1909-12-25 -1909-05-05 12:34:56.1047,1909-12-25 -1909-05-05 12:34:56.1048,1909-12-25 -1909-05-05 12:34:56.1049,1909-12-25 -1909-05-05 12:34:56.105,1909-12-25 -1909-05-05 12:34:56.1051,1909-12-25 -1909-05-05 12:34:56.1052,1909-12-25 -1909-05-05 12:34:56.1053,1909-12-25 -1909-05-05 12:34:56.1054,1909-12-25 -1909-05-05 12:34:56.1055,1909-12-25 -1909-05-05 12:34:56.1056,1909-12-25 -1909-05-05 12:34:56.1057,1909-12-25 -1909-05-05 12:34:56.1058,1909-12-25 -1909-05-05 12:34:56.1059,1909-12-25 -1909-05-05 12:34:56.106,1909-12-25 -1909-05-05 12:34:56.1061,1909-12-25 -1909-05-05 12:34:56.1062,1909-12-25 -1909-05-05 12:34:56.1063,1909-12-25 -1909-05-05 12:34:56.1064,1909-12-25 -1909-05-05 12:34:56.1065,1909-12-25 -1909-05-05 12:34:56.1066,1909-12-25 -1909-05-05 12:34:56.1067,1909-12-25 -1909-05-05 12:34:56.1068,1909-12-25 -1909-05-05 12:34:56.1069,1909-12-25 -1909-05-05 12:34:56.107,1909-12-25 -1909-05-05 12:34:56.1071,1909-12-25 -1909-05-05 12:34:56.1072,1909-12-25 -1909-05-05 12:34:56.1073,1909-12-25 -1909-05-05 12:34:56.1074,1909-12-25 -1909-05-05 12:34:56.1075,1909-12-25 -1909-05-05 12:34:56.1076,1909-12-25 -1909-05-05 12:34:56.1077,1909-12-25 -1909-05-05 12:34:56.1078,1909-12-25 -1909-05-05 12:34:56.1079,1909-12-25 -1909-05-05 12:34:56.108,1909-12-25 -1909-05-05 12:34:56.1081,1909-12-25 -1909-05-05 12:34:56.1082,1909-12-25 -1909-05-05 12:34:56.1083,1909-12-25 -1909-05-05 12:34:56.1084,1909-12-25 -1909-05-05 12:34:56.1085,1909-12-25 -1909-05-05 12:34:56.1086,1909-12-25 -1909-05-05 12:34:56.1087,1909-12-25 -1909-05-05 12:34:56.1088,1909-12-25 -1909-05-05 12:34:56.1089,1909-12-25 -1909-05-05 12:34:56.109,1909-12-25 -1909-05-05 12:34:56.1091,1909-12-25 -1909-05-05 12:34:56.1092,1909-12-25 -1909-05-05 12:34:56.1093,1909-12-25 -1909-05-05 12:34:56.1094,1909-12-25 -1909-05-05 12:34:56.1095,1909-12-25 -1909-05-05 12:34:56.1096,1909-12-25 -1909-05-05 12:34:56.1097,1909-12-25 -1909-05-05 12:34:56.1098,1909-12-25 -1909-05-05 12:34:56.1099,1909-12-25 -1909-05-05 12:34:56.11,1909-12-25 -1909-05-05 12:34:56.1101,1909-12-25 -1909-05-05 12:34:56.1102,1909-12-25 -1909-05-05 12:34:56.1103,1909-12-25 -1909-05-05 12:34:56.1104,1909-12-25 -1909-05-05 12:34:56.1105,1909-12-25 -1909-05-05 12:34:56.1106,1909-12-25 -1909-05-05 12:34:56.1107,1909-12-25 -1909-05-05 12:34:56.1108,1909-12-25 -1909-05-05 12:34:56.1109,1909-12-25 -1909-05-05 12:34:56.111,1909-12-25 -1909-05-05 12:34:56.1111,1909-12-25 -1909-05-05 12:34:56.1112,1909-12-25 -1909-05-05 12:34:56.1113,1909-12-25 -1909-05-05 12:34:56.1114,1909-12-25 -1909-05-05 12:34:56.1115,1909-12-25 -1909-05-05 12:34:56.1116,1909-12-25 -1909-05-05 12:34:56.1117,1909-12-25 -1909-05-05 12:34:56.1118,1909-12-25 -1909-05-05 12:34:56.1119,1909-12-25 -1909-05-05 12:34:56.112,1909-12-25 -1909-05-05 12:34:56.1121,1909-12-25 -1909-05-05 12:34:56.1122,1909-12-25 -1909-05-05 12:34:56.1123,1909-12-25 -1909-05-05 12:34:56.1124,1909-12-25 -1909-05-05 12:34:56.1125,1909-12-25 -1909-05-05 12:34:56.1126,1909-12-25 -1909-05-05 12:34:56.1127,1909-12-25 -1909-05-05 12:34:56.1128,1909-12-25 -1909-05-05 12:34:56.1129,1909-12-25 -1909-05-05 12:34:56.113,1909-12-25 -1909-05-05 12:34:56.1131,1909-12-25 -1909-05-05 12:34:56.1132,1909-12-25 -1909-05-05 12:34:56.1133,1909-12-25 -1909-05-05 12:34:56.1134,1909-12-25 -1909-05-05 12:34:56.1135,1909-12-25 -1909-05-05 12:34:56.1136,1909-12-25 -1909-05-05 12:34:56.1137,1909-12-25 -1909-05-05 12:34:56.1138,1909-12-25 -1909-05-05 12:34:56.1139,1909-12-25 -1909-05-05 12:34:56.114,1909-12-25 -1909-05-05 12:34:56.1141,1909-12-25 -1909-05-05 12:34:56.1142,1909-12-25 -1909-05-05 12:34:56.1143,1909-12-25 -1909-05-05 12:34:56.1144,1909-12-25 -1909-05-05 12:34:56.1145,1909-12-25 -1909-05-05 12:34:56.1146,1909-12-25 -1909-05-05 12:34:56.1147,1909-12-25 -1909-05-05 12:34:56.1148,1909-12-25 -1909-05-05 12:34:56.1149,1909-12-25 -1909-05-05 12:34:56.115,1909-12-25 -1909-05-05 12:34:56.1151,1909-12-25 -1909-05-05 12:34:56.1152,1909-12-25 -1909-05-05 12:34:56.1153,1909-12-25 -1909-05-05 12:34:56.1154,1909-12-25 -1909-05-05 12:34:56.1155,1909-12-25 -1909-05-05 12:34:56.1156,1909-12-25 -1909-05-05 12:34:56.1157,1909-12-25 -1909-05-05 12:34:56.1158,1909-12-25 -1909-05-05 12:34:56.1159,1909-12-25 -1909-05-05 12:34:56.116,1909-12-25 -1909-05-05 12:34:56.1161,1909-12-25 -1909-05-05 12:34:56.1162,1909-12-25 -1909-05-05 12:34:56.1163,1909-12-25 -1909-05-05 12:34:56.1164,1909-12-25 -1909-05-05 12:34:56.1165,1909-12-25 -1909-05-05 12:34:56.1166,1909-12-25 -1909-05-05 12:34:56.1167,1909-12-25 -1909-05-05 12:34:56.1168,1909-12-25 -1909-05-05 12:34:56.1169,1909-12-25 -1909-05-05 12:34:56.117,1909-12-25 -1909-05-05 12:34:56.1171,1909-12-25 -1909-05-05 12:34:56.1172,1909-12-25 -1909-05-05 12:34:56.1173,1909-12-25 -1909-05-05 12:34:56.1174,1909-12-25 -1909-05-05 12:34:56.1175,1909-12-25 -1909-05-05 12:34:56.1176,1909-12-25 -1909-05-05 12:34:56.1177,1909-12-25 -1909-05-05 12:34:56.1178,1909-12-25 -1909-05-05 12:34:56.1179,1909-12-25 -1909-05-05 12:34:56.118,1909-12-25 -1909-05-05 12:34:56.1181,1909-12-25 -1909-05-05 12:34:56.1182,1909-12-25 -1909-05-05 12:34:56.1183,1909-12-25 -1909-05-05 12:34:56.1184,1909-12-25 -1909-05-05 12:34:56.1185,1909-12-25 -1909-05-05 12:34:56.1186,1909-12-25 -1909-05-05 12:34:56.1187,1909-12-25 -1909-05-05 12:34:56.1188,1909-12-25 -1909-05-05 12:34:56.1189,1909-12-25 -1909-05-05 12:34:56.119,1909-12-25 -1909-05-05 12:34:56.1191,1909-12-25 -1909-05-05 12:34:56.1192,1909-12-25 -1909-05-05 12:34:56.1193,1909-12-25 -1909-05-05 12:34:56.1194,1909-12-25 -1909-05-05 12:34:56.1195,1909-12-25 -1909-05-05 12:34:56.1196,1909-12-25 -1909-05-05 12:34:56.1197,1909-12-25 -1909-05-05 12:34:56.1198,1909-12-25 -1909-05-05 12:34:56.1199,1909-12-25 -1909-05-05 12:34:56.12,1909-12-25 -1909-05-05 12:34:56.1201,1909-12-25 -1909-05-05 12:34:56.1202,1909-12-25 -1909-05-05 12:34:56.1203,1909-12-25 -1909-05-05 12:34:56.1204,1909-12-25 -1909-05-05 12:34:56.1205,1909-12-25 -1909-05-05 12:34:56.1206,1909-12-25 -1909-05-05 12:34:56.1207,1909-12-25 -1909-05-05 12:34:56.1208,1909-12-25 -1909-05-05 12:34:56.1209,1909-12-25 -1909-05-05 12:34:56.121,1909-12-25 -1909-05-05 12:34:56.1211,1909-12-25 -1909-05-05 12:34:56.1212,1909-12-25 -1909-05-05 12:34:56.1213,1909-12-25 -1909-05-05 12:34:56.1214,1909-12-25 -1909-05-05 12:34:56.1215,1909-12-25 -1909-05-05 12:34:56.1216,1909-12-25 -1909-05-05 12:34:56.1217,1909-12-25 -1909-05-05 12:34:56.1218,1909-12-25 -1909-05-05 12:34:56.1219,1909-12-25 -1909-05-05 12:34:56.122,1909-12-25 -1909-05-05 12:34:56.1221,1909-12-25 -1909-05-05 12:34:56.1222,1909-12-25 -1909-05-05 12:34:56.1223,1909-12-25 -1909-05-05 12:34:56.1224,1909-12-25 -1909-05-05 12:34:56.1225,1909-12-25 -1909-05-05 12:34:56.1226,1909-12-25 -1909-05-05 12:34:56.1227,1909-12-25 -1909-05-05 12:34:56.1228,1909-12-25 -1909-05-05 12:34:56.1229,1909-12-25 -1909-05-05 12:34:56.123,1909-12-25 -1909-05-05 12:34:56.1231,1909-12-25 -1909-05-05 12:34:56.1232,1909-12-25 -1909-05-05 12:34:56.1233,1909-12-25 -1909-05-05 12:34:56.1234,1909-12-25 -1909-05-05 12:34:56.1235,1909-12-25 -1909-05-05 12:34:56.1236,1909-12-25 -1909-05-05 12:34:56.1237,1909-12-25 -1909-05-05 12:34:56.1238,1909-12-25 -1909-05-05 12:34:56.1239,1909-12-25 -1909-05-05 12:34:56.124,1909-12-25 -1909-05-05 12:34:56.1241,1909-12-25 -1909-05-05 12:34:56.1242,1909-12-25 -1909-05-05 12:34:56.1243,1909-12-25 -1909-05-05 12:34:56.1244,1909-12-25 -1909-05-05 12:34:56.1245,1909-12-25 -1909-05-05 12:34:56.1246,1909-12-25 -1909-05-05 12:34:56.1247,1909-12-25 -1909-05-05 12:34:56.1248,1909-12-25 -1909-05-05 12:34:56.1249,1909-12-25 -1909-05-05 12:34:56.125,1909-12-25 -1909-05-05 12:34:56.1251,1909-12-25 -1909-05-05 12:34:56.1252,1909-12-25 -1909-05-05 12:34:56.1253,1909-12-25 -1909-05-05 12:34:56.1254,1909-12-25 -1909-05-05 12:34:56.1255,1909-12-25 -1909-05-05 12:34:56.1256,1909-12-25 -1909-05-05 12:34:56.1257,1909-12-25 -1909-05-05 12:34:56.1258,1909-12-25 -1909-05-05 12:34:56.1259,1909-12-25 -1909-05-05 12:34:56.126,1909-12-25 -1909-05-05 12:34:56.1261,1909-12-25 -1909-05-05 12:34:56.1262,1909-12-25 -1909-05-05 12:34:56.1263,1909-12-25 -1909-05-05 12:34:56.1264,1909-12-25 -1909-05-05 12:34:56.1265,1909-12-25 -1909-05-05 12:34:56.1266,1909-12-25 -1909-05-05 12:34:56.1267,1909-12-25 -1909-05-05 12:34:56.1268,1909-12-25 -1909-05-05 12:34:56.1269,1909-12-25 -1909-05-05 12:34:56.127,1909-12-25 -1909-05-05 12:34:56.1271,1909-12-25 -1909-05-05 12:34:56.1272,1909-12-25 -1909-05-05 12:34:56.1273,1909-12-25 -1909-05-05 12:34:56.1274,1909-12-25 -1909-05-05 12:34:56.1275,1909-12-25 -1909-05-05 12:34:56.1276,1909-12-25 -1909-05-05 12:34:56.1277,1909-12-25 -1909-05-05 12:34:56.1278,1909-12-25 -1909-05-05 12:34:56.1279,1909-12-25 -1909-05-05 12:34:56.128,1909-12-25 -1909-05-05 12:34:56.1281,1909-12-25 -1909-05-05 12:34:56.1282,1909-12-25 -1909-05-05 12:34:56.1283,1909-12-25 -1909-05-05 12:34:56.1284,1909-12-25 -1909-05-05 12:34:56.1285,1909-12-25 -1909-05-05 12:34:56.1286,1909-12-25 -1909-05-05 12:34:56.1287,1909-12-25 -1909-05-05 12:34:56.1288,1909-12-25 -1909-05-05 12:34:56.1289,1909-12-25 -1909-05-05 12:34:56.129,1909-12-25 -1909-05-05 12:34:56.1291,1909-12-25 -1909-05-05 12:34:56.1292,1909-12-25 -1909-05-05 12:34:56.1293,1909-12-25 -1909-05-05 12:34:56.1294,1909-12-25 -1909-05-05 12:34:56.1295,1909-12-25 -1909-05-05 12:34:56.1296,1909-12-25 -1909-05-05 12:34:56.1297,1909-12-25 -1909-05-05 12:34:56.1298,1909-12-25 -1909-05-05 12:34:56.1299,1909-12-25 -1909-05-05 12:34:56.13,1909-12-25 -1909-05-05 12:34:56.1301,1909-12-25 -1909-05-05 12:34:56.1302,1909-12-25 -1909-05-05 12:34:56.1303,1909-12-25 -1909-05-05 12:34:56.1304,1909-12-25 -1909-05-05 12:34:56.1305,1909-12-25 -1909-05-05 12:34:56.1306,1909-12-25 -1909-05-05 12:34:56.1307,1909-12-25 -1909-05-05 12:34:56.1308,1909-12-25 -1909-05-05 12:34:56.1309,1909-12-25 -1909-05-05 12:34:56.131,1909-12-25 -1909-05-05 12:34:56.1311,1909-12-25 -1909-05-05 12:34:56.1312,1909-12-25 -1909-05-05 12:34:56.1313,1909-12-25 -1909-05-05 12:34:56.1314,1909-12-25 -1909-05-05 12:34:56.1315,1909-12-25 -1909-05-05 12:34:56.1316,1909-12-25 -1909-05-05 12:34:56.1317,1909-12-25 -1909-05-05 12:34:56.1318,1909-12-25 -1909-05-05 12:34:56.1319,1909-12-25 -1909-05-05 12:34:56.132,1909-12-25 -1909-05-05 12:34:56.1321,1909-12-25 -1909-05-05 12:34:56.1322,1909-12-25 -1909-05-05 12:34:56.1323,1909-12-25 -1909-05-05 12:34:56.1324,1909-12-25 -1909-05-05 12:34:56.1325,1909-12-25 -1909-05-05 12:34:56.1326,1909-12-25 -1909-05-05 12:34:56.1327,1909-12-25 -1909-05-05 12:34:56.1328,1909-12-25 -1909-05-05 12:34:56.1329,1909-12-25 -1909-05-05 12:34:56.133,1909-12-25 -1909-05-05 12:34:56.1331,1909-12-25 -1909-05-05 12:34:56.1332,1909-12-25 -1909-05-05 12:34:56.1333,1909-12-25 -1909-05-05 12:34:56.1334,1909-12-25 -1909-05-05 12:34:56.1335,1909-12-25 -1909-05-05 12:34:56.1336,1909-12-25 -1909-05-05 12:34:56.1337,1909-12-25 -1909-05-05 12:34:56.1338,1909-12-25 -1909-05-05 12:34:56.1339,1909-12-25 -1909-05-05 12:34:56.134,1909-12-25 -1909-05-05 12:34:56.1341,1909-12-25 -1909-05-05 12:34:56.1342,1909-12-25 -1909-05-05 12:34:56.1343,1909-12-25 -1909-05-05 12:34:56.1344,1909-12-25 -1909-05-05 12:34:56.1345,1909-12-25 -1909-05-05 12:34:56.1346,1909-12-25 -1909-05-05 12:34:56.1347,1909-12-25 -1909-05-05 12:34:56.1348,1909-12-25 -1909-05-05 12:34:56.1349,1909-12-25 -1909-05-05 12:34:56.135,1909-12-25 -1909-05-05 12:34:56.1351,1909-12-25 -1909-05-05 12:34:56.1352,1909-12-25 -1909-05-05 12:34:56.1353,1909-12-25 -1909-05-05 12:34:56.1354,1909-12-25 -1909-05-05 12:34:56.1355,1909-12-25 -1909-05-05 12:34:56.1356,1909-12-25 -1909-05-05 12:34:56.1357,1909-12-25 -1909-05-05 12:34:56.1358,1909-12-25 -1909-05-05 12:34:56.1359,1909-12-25 -1909-05-05 12:34:56.136,1909-12-25 -1909-05-05 12:34:56.1361,1909-12-25 -1909-05-05 12:34:56.1362,1909-12-25 -1909-05-05 12:34:56.1363,1909-12-25 -1909-05-05 12:34:56.1364,1909-12-25 -1909-05-05 12:34:56.1365,1909-12-25 -1909-05-05 12:34:56.1366,1909-12-25 -1909-05-05 12:34:56.1367,1909-12-25 -1909-05-05 12:34:56.1368,1909-12-25 -1909-05-05 12:34:56.1369,1909-12-25 -1909-05-05 12:34:56.137,1909-12-25 -1909-05-05 12:34:56.1371,1909-12-25 -1909-05-05 12:34:56.1372,1909-12-25 -1909-05-05 12:34:56.1373,1909-12-25 -1909-05-05 12:34:56.1374,1909-12-25 -1909-05-05 12:34:56.1375,1909-12-25 -1909-05-05 12:34:56.1376,1909-12-25 -1909-05-05 12:34:56.1377,1909-12-25 -1909-05-05 12:34:56.1378,1909-12-25 -1909-05-05 12:34:56.1379,1909-12-25 -1909-05-05 12:34:56.138,1909-12-25 -1909-05-05 12:34:56.1381,1909-12-25 -1909-05-05 12:34:56.1382,1909-12-25 -1909-05-05 12:34:56.1383,1909-12-25 -1909-05-05 12:34:56.1384,1909-12-25 -1909-05-05 12:34:56.1385,1909-12-25 -1909-05-05 12:34:56.1386,1909-12-25 -1909-05-05 12:34:56.1387,1909-12-25 -1909-05-05 12:34:56.1388,1909-12-25 -1909-05-05 12:34:56.1389,1909-12-25 -1909-05-05 12:34:56.139,1909-12-25 -1909-05-05 12:34:56.1391,1909-12-25 -1909-05-05 12:34:56.1392,1909-12-25 -1909-05-05 12:34:56.1393,1909-12-25 -1909-05-05 12:34:56.1394,1909-12-25 -1909-05-05 12:34:56.1395,1909-12-25 -1909-05-05 12:34:56.1396,1909-12-25 -1909-05-05 12:34:56.1397,1909-12-25 -1909-05-05 12:34:56.1398,1909-12-25 -1909-05-05 12:34:56.1399,1909-12-25 -1909-05-05 12:34:56.14,1909-12-25 -1909-05-05 12:34:56.1401,1909-12-25 -1909-05-05 12:34:56.1402,1909-12-25 -1909-05-05 12:34:56.1403,1909-12-25 -1909-05-05 12:34:56.1404,1909-12-25 -1909-05-05 12:34:56.1405,1909-12-25 -1909-05-05 12:34:56.1406,1909-12-25 -1909-05-05 12:34:56.1407,1909-12-25 -1909-05-05 12:34:56.1408,1909-12-25 -1909-05-05 12:34:56.1409,1909-12-25 -1909-05-05 12:34:56.141,1909-12-25 -1909-05-05 12:34:56.1411,1909-12-25 -1909-05-05 12:34:56.1412,1909-12-25 -1909-05-05 12:34:56.1413,1909-12-25 -1909-05-05 12:34:56.1414,1909-12-25 -1909-05-05 12:34:56.1415,1909-12-25 -1909-05-05 12:34:56.1416,1909-12-25 -1909-05-05 12:34:56.1417,1909-12-25 -1909-05-05 12:34:56.1418,1909-12-25 -1909-05-05 12:34:56.1419,1909-12-25 -1909-05-05 12:34:56.142,1909-12-25 -1909-05-05 12:34:56.1421,1909-12-25 -1909-05-05 12:34:56.1422,1909-12-25 -1909-05-05 12:34:56.1423,1909-12-25 -1909-05-05 12:34:56.1424,1909-12-25 -1909-05-05 12:34:56.1425,1909-12-25 -1909-05-05 12:34:56.1426,1909-12-25 -1909-05-05 12:34:56.1427,1909-12-25 -1909-05-05 12:34:56.1428,1909-12-25 -1909-05-05 12:34:56.1429,1909-12-25 -1909-05-05 12:34:56.143,1909-12-25 -1909-05-05 12:34:56.1431,1909-12-25 -1909-05-05 12:34:56.1432,1909-12-25 -1909-05-05 12:34:56.1433,1909-12-25 -1909-05-05 12:34:56.1434,1909-12-25 -1909-05-05 12:34:56.1435,1909-12-25 -1909-05-05 12:34:56.1436,1909-12-25 -1909-05-05 12:34:56.1437,1909-12-25 -1909-05-05 12:34:56.1438,1909-12-25 -1909-05-05 12:34:56.1439,1909-12-25 -1909-05-05 12:34:56.144,1909-12-25 -1909-05-05 12:34:56.1441,1909-12-25 -1909-05-05 12:34:56.1442,1909-12-25 -1909-05-05 12:34:56.1443,1909-12-25 -1909-05-05 12:34:56.1444,1909-12-25 -1909-05-05 12:34:56.1445,1909-12-25 -1909-05-05 12:34:56.1446,1909-12-25 -1909-05-05 12:34:56.1447,1909-12-25 -1909-05-05 12:34:56.1448,1909-12-25 -1909-05-05 12:34:56.1449,1909-12-25 -1909-05-05 12:34:56.145,1909-12-25 -1909-05-05 12:34:56.1451,1909-12-25 -1909-05-05 12:34:56.1452,1909-12-25 -1909-05-05 12:34:56.1453,1909-12-25 -1909-05-05 12:34:56.1454,1909-12-25 -1909-05-05 12:34:56.1455,1909-12-25 -1909-05-05 12:34:56.1456,1909-12-25 -1909-05-05 12:34:56.1457,1909-12-25 -1909-05-05 12:34:56.1458,1909-12-25 -1909-05-05 12:34:56.1459,1909-12-25 -1909-05-05 12:34:56.146,1909-12-25 -1909-05-05 12:34:56.1461,1909-12-25 -1909-05-05 12:34:56.1462,1909-12-25 -1909-05-05 12:34:56.1463,1909-12-25 -1909-05-05 12:34:56.1464,1909-12-25 -1909-05-05 12:34:56.1465,1909-12-25 -1909-05-05 12:34:56.1466,1909-12-25 -1909-05-05 12:34:56.1467,1909-12-25 -1909-05-05 12:34:56.1468,1909-12-25 -1909-05-05 12:34:56.1469,1909-12-25 -1909-05-05 12:34:56.147,1909-12-25 -1909-05-05 12:34:56.1471,1909-12-25 -1909-05-05 12:34:56.1472,1909-12-25 -1909-05-05 12:34:56.1473,1909-12-25 -1909-05-05 12:34:56.1474,1909-12-25 -1909-05-05 12:34:56.1475,1909-12-25 -1909-05-05 12:34:56.1476,1909-12-25 -1909-05-05 12:34:56.1477,1909-12-25 -1909-05-05 12:34:56.1478,1909-12-25 -1909-05-05 12:34:56.1479,1909-12-25 -1909-05-05 12:34:56.148,1909-12-25 -1909-05-05 12:34:56.1481,1909-12-25 -1909-05-05 12:34:56.1482,1909-12-25 -1909-05-05 12:34:56.1483,1909-12-25 -1909-05-05 12:34:56.1484,1909-12-25 -1909-05-05 12:34:56.1485,1909-12-25 -1909-05-05 12:34:56.1486,1909-12-25 -1909-05-05 12:34:56.1487,1909-12-25 -1909-05-05 12:34:56.1488,1909-12-25 -1909-05-05 12:34:56.1489,1909-12-25 -1909-05-05 12:34:56.149,1909-12-25 -1909-05-05 12:34:56.1491,1909-12-25 -1909-05-05 12:34:56.1492,1909-12-25 -1909-05-05 12:34:56.1493,1909-12-25 -1909-05-05 12:34:56.1494,1909-12-25 -1909-05-05 12:34:56.1495,1909-12-25 -1909-05-05 12:34:56.1496,1909-12-25 -1909-05-05 12:34:56.1497,1909-12-25 -1909-05-05 12:34:56.1498,1909-12-25 -1909-05-05 12:34:56.1499,1909-12-25 -1909-05-05 12:34:56.15,1909-12-25 -1909-05-05 12:34:56.1501,1909-12-25 -1909-05-05 12:34:56.1502,1909-12-25 -1909-05-05 12:34:56.1503,1909-12-25 -1909-05-05 12:34:56.1504,1909-12-25 -1909-05-05 12:34:56.1505,1909-12-25 -1909-05-05 12:34:56.1506,1909-12-25 -1909-05-05 12:34:56.1507,1909-12-25 -1909-05-05 12:34:56.1508,1909-12-25 -1909-05-05 12:34:56.1509,1909-12-25 -1909-05-05 12:34:56.151,1909-12-25 -1909-05-05 12:34:56.1511,1909-12-25 -1909-05-05 12:34:56.1512,1909-12-25 -1909-05-05 12:34:56.1513,1909-12-25 -1909-05-05 12:34:56.1514,1909-12-25 -1909-05-05 12:34:56.1515,1909-12-25 -1909-05-05 12:34:56.1516,1909-12-25 -1909-05-05 12:34:56.1517,1909-12-25 -1909-05-05 12:34:56.1518,1909-12-25 -1909-05-05 12:34:56.1519,1909-12-25 -1909-05-05 12:34:56.152,1909-12-25 -1909-05-05 12:34:56.1521,1909-12-25 -1909-05-05 12:34:56.1522,1909-12-25 -1909-05-05 12:34:56.1523,1909-12-25 -1909-05-05 12:34:56.1524,1909-12-25 -1909-05-05 12:34:56.1525,1909-12-25 -1909-05-05 12:34:56.1526,1909-12-25 -1909-05-05 12:34:56.1527,1909-12-25 -1909-05-05 12:34:56.1528,1909-12-25 -1909-05-05 12:34:56.1529,1909-12-25 -1909-05-05 12:34:56.153,1909-12-25 -1909-05-05 12:34:56.1531,1909-12-25 -1909-05-05 12:34:56.1532,1909-12-25 -1909-05-05 12:34:56.1533,1909-12-25 -1909-05-05 12:34:56.1534,1909-12-25 -1909-05-05 12:34:56.1535,1909-12-25 -1909-05-05 12:34:56.1536,1909-12-25 -1909-05-05 12:34:56.1537,1909-12-25 -1909-05-05 12:34:56.1538,1909-12-25 -1909-05-05 12:34:56.1539,1909-12-25 -1909-05-05 12:34:56.154,1909-12-25 -1909-05-05 12:34:56.1541,1909-12-25 -1909-05-05 12:34:56.1542,1909-12-25 -1909-05-05 12:34:56.1543,1909-12-25 -1909-05-05 12:34:56.1544,1909-12-25 -1909-05-05 12:34:56.1545,1909-12-25 -1909-05-05 12:34:56.1546,1909-12-25 -1909-05-05 12:34:56.1547,1909-12-25 -1909-05-05 12:34:56.1548,1909-12-25 -1909-05-05 12:34:56.1549,1909-12-25 -1909-05-05 12:34:56.155,1909-12-25 -1909-05-05 12:34:56.1551,1909-12-25 -1909-05-05 12:34:56.1552,1909-12-25 -1909-05-05 12:34:56.1553,1909-12-25 -1909-05-05 12:34:56.1554,1909-12-25 -1909-05-05 12:34:56.1555,1909-12-25 -1909-05-05 12:34:56.1556,1909-12-25 -1909-05-05 12:34:56.1557,1909-12-25 -1909-05-05 12:34:56.1558,1909-12-25 -1909-05-05 12:34:56.1559,1909-12-25 -1909-05-05 12:34:56.156,1909-12-25 -1909-05-05 12:34:56.1561,1909-12-25 -1909-05-05 12:34:56.1562,1909-12-25 -1909-05-05 12:34:56.1563,1909-12-25 -1909-05-05 12:34:56.1564,1909-12-25 -1909-05-05 12:34:56.1565,1909-12-25 -1909-05-05 12:34:56.1566,1909-12-25 -1909-05-05 12:34:56.1567,1909-12-25 -1909-05-05 12:34:56.1568,1909-12-25 -1909-05-05 12:34:56.1569,1909-12-25 -1909-05-05 12:34:56.157,1909-12-25 -1909-05-05 12:34:56.1571,1909-12-25 -1909-05-05 12:34:56.1572,1909-12-25 -1909-05-05 12:34:56.1573,1909-12-25 -1909-05-05 12:34:56.1574,1909-12-25 -1909-05-05 12:34:56.1575,1909-12-25 -1909-05-05 12:34:56.1576,1909-12-25 -1909-05-05 12:34:56.1577,1909-12-25 -1909-05-05 12:34:56.1578,1909-12-25 -1909-05-05 12:34:56.1579,1909-12-25 -1909-05-05 12:34:56.158,1909-12-25 -1909-05-05 12:34:56.1581,1909-12-25 -1909-05-05 12:34:56.1582,1909-12-25 -1909-05-05 12:34:56.1583,1909-12-25 -1909-05-05 12:34:56.1584,1909-12-25 -1909-05-05 12:34:56.1585,1909-12-25 -1909-05-05 12:34:56.1586,1909-12-25 -1909-05-05 12:34:56.1587,1909-12-25 -1909-05-05 12:34:56.1588,1909-12-25 -1909-05-05 12:34:56.1589,1909-12-25 -1909-05-05 12:34:56.159,1909-12-25 -1909-05-05 12:34:56.1591,1909-12-25 -1909-05-05 12:34:56.1592,1909-12-25 -1909-05-05 12:34:56.1593,1909-12-25 -1909-05-05 12:34:56.1594,1909-12-25 -1909-05-05 12:34:56.1595,1909-12-25 -1909-05-05 12:34:56.1596,1909-12-25 -1909-05-05 12:34:56.1597,1909-12-25 -1909-05-05 12:34:56.1598,1909-12-25 -1909-05-05 12:34:56.1599,1909-12-25 -1909-05-05 12:34:56.16,1909-12-25 -1909-05-05 12:34:56.1601,1909-12-25 -1909-05-05 12:34:56.1602,1909-12-25 -1909-05-05 12:34:56.1603,1909-12-25 -1909-05-05 12:34:56.1604,1909-12-25 -1909-05-05 12:34:56.1605,1909-12-25 -1909-05-05 12:34:56.1606,1909-12-25 -1909-05-05 12:34:56.1607,1909-12-25 -1909-05-05 12:34:56.1608,1909-12-25 -1909-05-05 12:34:56.1609,1909-12-25 -1909-05-05 12:34:56.161,1909-12-25 -1909-05-05 12:34:56.1611,1909-12-25 -1909-05-05 12:34:56.1612,1909-12-25 -1909-05-05 12:34:56.1613,1909-12-25 -1909-05-05 12:34:56.1614,1909-12-25 -1909-05-05 12:34:56.1615,1909-12-25 -1909-05-05 12:34:56.1616,1909-12-25 -1909-05-05 12:34:56.1617,1909-12-25 -1909-05-05 12:34:56.1618,1909-12-25 -1909-05-05 12:34:56.1619,1909-12-25 -1909-05-05 12:34:56.162,1909-12-25 -1909-05-05 12:34:56.1621,1909-12-25 -1909-05-05 12:34:56.1622,1909-12-25 -1909-05-05 12:34:56.1623,1909-12-25 -1909-05-05 12:34:56.1624,1909-12-25 -1909-05-05 12:34:56.1625,1909-12-25 -1909-05-05 12:34:56.1626,1909-12-25 -1909-05-05 12:34:56.1627,1909-12-25 -1909-05-05 12:34:56.1628,1909-12-25 -1909-05-05 12:34:56.1629,1909-12-25 -1909-05-05 12:34:56.163,1909-12-25 -1909-05-05 12:34:56.1631,1909-12-25 -1909-05-05 12:34:56.1632,1909-12-25 -1909-05-05 12:34:56.1633,1909-12-25 -1909-05-05 12:34:56.1634,1909-12-25 -1909-05-05 12:34:56.1635,1909-12-25 -1909-05-05 12:34:56.1636,1909-12-25 -1909-05-05 12:34:56.1637,1909-12-25 -1909-05-05 12:34:56.1638,1909-12-25 -1909-05-05 12:34:56.1639,1909-12-25 -1909-05-05 12:34:56.164,1909-12-25 -1909-05-05 12:34:56.1641,1909-12-25 -1909-05-05 12:34:56.1642,1909-12-25 -1909-05-05 12:34:56.1643,1909-12-25 -1909-05-05 12:34:56.1644,1909-12-25 -1909-05-05 12:34:56.1645,1909-12-25 -1909-05-05 12:34:56.1646,1909-12-25 -1909-05-05 12:34:56.1647,1909-12-25 -1909-05-05 12:34:56.1648,1909-12-25 -1909-05-05 12:34:56.1649,1909-12-25 -1909-05-05 12:34:56.165,1909-12-25 -1909-05-05 12:34:56.1651,1909-12-25 -1909-05-05 12:34:56.1652,1909-12-25 -1909-05-05 12:34:56.1653,1909-12-25 -1909-05-05 12:34:56.1654,1909-12-25 -1909-05-05 12:34:56.1655,1909-12-25 -1909-05-05 12:34:56.1656,1909-12-25 -1909-05-05 12:34:56.1657,1909-12-25 -1909-05-05 12:34:56.1658,1909-12-25 -1909-05-05 12:34:56.1659,1909-12-25 -1909-05-05 12:34:56.166,1909-12-25 -1909-05-05 12:34:56.1661,1909-12-25 -1909-05-05 12:34:56.1662,1909-12-25 -1909-05-05 12:34:56.1663,1909-12-25 -1909-05-05 12:34:56.1664,1909-12-25 -1909-05-05 12:34:56.1665,1909-12-25 -1909-05-05 12:34:56.1666,1909-12-25 -1909-05-05 12:34:56.1667,1909-12-25 -1909-05-05 12:34:56.1668,1909-12-25 -1909-05-05 12:34:56.1669,1909-12-25 -1909-05-05 12:34:56.167,1909-12-25 -1909-05-05 12:34:56.1671,1909-12-25 -1909-05-05 12:34:56.1672,1909-12-25 -1909-05-05 12:34:56.1673,1909-12-25 -1909-05-05 12:34:56.1674,1909-12-25 -1909-05-05 12:34:56.1675,1909-12-25 -1909-05-05 12:34:56.1676,1909-12-25 -1909-05-05 12:34:56.1677,1909-12-25 -1909-05-05 12:34:56.1678,1909-12-25 -1909-05-05 12:34:56.1679,1909-12-25 -1909-05-05 12:34:56.168,1909-12-25 -1909-05-05 12:34:56.1681,1909-12-25 -1909-05-05 12:34:56.1682,1909-12-25 -1909-05-05 12:34:56.1683,1909-12-25 -1909-05-05 12:34:56.1684,1909-12-25 -1909-05-05 12:34:56.1685,1909-12-25 -1909-05-05 12:34:56.1686,1909-12-25 -1909-05-05 12:34:56.1687,1909-12-25 -1909-05-05 12:34:56.1688,1909-12-25 -1909-05-05 12:34:56.1689,1909-12-25 -1909-05-05 12:34:56.169,1909-12-25 -1909-05-05 12:34:56.1691,1909-12-25 -1909-05-05 12:34:56.1692,1909-12-25 -1909-05-05 12:34:56.1693,1909-12-25 -1909-05-05 12:34:56.1694,1909-12-25 -1909-05-05 12:34:56.1695,1909-12-25 -1909-05-05 12:34:56.1696,1909-12-25 -1909-05-05 12:34:56.1697,1909-12-25 -1909-05-05 12:34:56.1698,1909-12-25 -1909-05-05 12:34:56.1699,1909-12-25 -1909-05-05 12:34:56.17,1909-12-25 -1909-05-05 12:34:56.1701,1909-12-25 -1909-05-05 12:34:56.1702,1909-12-25 -1909-05-05 12:34:56.1703,1909-12-25 -1909-05-05 12:34:56.1704,1909-12-25 -1909-05-05 12:34:56.1705,1909-12-25 -1909-05-05 12:34:56.1706,1909-12-25 -1909-05-05 12:34:56.1707,1909-12-25 -1909-05-05 12:34:56.1708,1909-12-25 -1909-05-05 12:34:56.1709,1909-12-25 -1909-05-05 12:34:56.171,1909-12-25 -1909-05-05 12:34:56.1711,1909-12-25 -1909-05-05 12:34:56.1712,1909-12-25 -1909-05-05 12:34:56.1713,1909-12-25 -1909-05-05 12:34:56.1714,1909-12-25 -1909-05-05 12:34:56.1715,1909-12-25 -1909-05-05 12:34:56.1716,1909-12-25 -1909-05-05 12:34:56.1717,1909-12-25 -1909-05-05 12:34:56.1718,1909-12-25 -1909-05-05 12:34:56.1719,1909-12-25 -1909-05-05 12:34:56.172,1909-12-25 -1909-05-05 12:34:56.1721,1909-12-25 -1909-05-05 12:34:56.1722,1909-12-25 -1909-05-05 12:34:56.1723,1909-12-25 -1909-05-05 12:34:56.1724,1909-12-25 -1909-05-05 12:34:56.1725,1909-12-25 -1909-05-05 12:34:56.1726,1909-12-25 -1909-05-05 12:34:56.1727,1909-12-25 -1909-05-05 12:34:56.1728,1909-12-25 -1909-05-05 12:34:56.1729,1909-12-25 -1909-05-05 12:34:56.173,1909-12-25 -1909-05-05 12:34:56.1731,1909-12-25 -1909-05-05 12:34:56.1732,1909-12-25 -1909-05-05 12:34:56.1733,1909-12-25 -1909-05-05 12:34:56.1734,1909-12-25 -1909-05-05 12:34:56.1735,1909-12-25 -1909-05-05 12:34:56.1736,1909-12-25 -1909-05-05 12:34:56.1737,1909-12-25 -1909-05-05 12:34:56.1738,1909-12-25 -1909-05-05 12:34:56.1739,1909-12-25 -1909-05-05 12:34:56.174,1909-12-25 -1909-05-05 12:34:56.1741,1909-12-25 -1909-05-05 12:34:56.1742,1909-12-25 -1909-05-05 12:34:56.1743,1909-12-25 -1909-05-05 12:34:56.1744,1909-12-25 -1909-05-05 12:34:56.1745,1909-12-25 -1909-05-05 12:34:56.1746,1909-12-25 -1909-05-05 12:34:56.1747,1909-12-25 -1909-05-05 12:34:56.1748,1909-12-25 -1909-05-05 12:34:56.1749,1909-12-25 -1909-05-05 12:34:56.175,1909-12-25 -1909-05-05 12:34:56.1751,1909-12-25 -1909-05-05 12:34:56.1752,1909-12-25 -1909-05-05 12:34:56.1753,1909-12-25 -1909-05-05 12:34:56.1754,1909-12-25 -1909-05-05 12:34:56.1755,1909-12-25 -1909-05-05 12:34:56.1756,1909-12-25 -1909-05-05 12:34:56.1757,1909-12-25 -1909-05-05 12:34:56.1758,1909-12-25 -1909-05-05 12:34:56.1759,1909-12-25 -1909-05-05 12:34:56.176,1909-12-25 -1909-05-05 12:34:56.1761,1909-12-25 -1909-05-05 12:34:56.1762,1909-12-25 -1909-05-05 12:34:56.1763,1909-12-25 -1909-05-05 12:34:56.1764,1909-12-25 -1909-05-05 12:34:56.1765,1909-12-25 -1909-05-05 12:34:56.1766,1909-12-25 -1909-05-05 12:34:56.1767,1909-12-25 -1909-05-05 12:34:56.1768,1909-12-25 -1909-05-05 12:34:56.1769,1909-12-25 -1909-05-05 12:34:56.177,1909-12-25 -1909-05-05 12:34:56.1771,1909-12-25 -1909-05-05 12:34:56.1772,1909-12-25 -1909-05-05 12:34:56.1773,1909-12-25 -1909-05-05 12:34:56.1774,1909-12-25 -1909-05-05 12:34:56.1775,1909-12-25 -1909-05-05 12:34:56.1776,1909-12-25 -1909-05-05 12:34:56.1777,1909-12-25 -1909-05-05 12:34:56.1778,1909-12-25 -1909-05-05 12:34:56.1779,1909-12-25 -1909-05-05 12:34:56.178,1909-12-25 -1909-05-05 12:34:56.1781,1909-12-25 -1909-05-05 12:34:56.1782,1909-12-25 -1909-05-05 12:34:56.1783,1909-12-25 -1909-05-05 12:34:56.1784,1909-12-25 -1909-05-05 12:34:56.1785,1909-12-25 -1909-05-05 12:34:56.1786,1909-12-25 -1909-05-05 12:34:56.1787,1909-12-25 -1909-05-05 12:34:56.1788,1909-12-25 -1909-05-05 12:34:56.1789,1909-12-25 -1909-05-05 12:34:56.179,1909-12-25 -1909-05-05 12:34:56.1791,1909-12-25 -1909-05-05 12:34:56.1792,1909-12-25 -1909-05-05 12:34:56.1793,1909-12-25 -1909-05-05 12:34:56.1794,1909-12-25 -1909-05-05 12:34:56.1795,1909-12-25 -1909-05-05 12:34:56.1796,1909-12-25 -1909-05-05 12:34:56.1797,1909-12-25 -1909-05-05 12:34:56.1798,1909-12-25 -1909-05-05 12:34:56.1799,1909-12-25 -1909-05-05 12:34:56.18,1909-12-25 -1909-05-05 12:34:56.1801,1909-12-25 -1909-05-05 12:34:56.1802,1909-12-25 -1909-05-05 12:34:56.1803,1909-12-25 -1909-05-05 12:34:56.1804,1909-12-25 -1909-05-05 12:34:56.1805,1909-12-25 -1909-05-05 12:34:56.1806,1909-12-25 -1909-05-05 12:34:56.1807,1909-12-25 -1909-05-05 12:34:56.1808,1909-12-25 -1909-05-05 12:34:56.1809,1909-12-25 -1909-05-05 12:34:56.181,1909-12-25 -1909-05-05 12:34:56.1811,1909-12-25 -1909-05-05 12:34:56.1812,1909-12-25 -1909-05-05 12:34:56.1813,1909-12-25 -1909-05-05 12:34:56.1814,1909-12-25 -1909-05-05 12:34:56.1815,1909-12-25 -1909-05-05 12:34:56.1816,1909-12-25 -1909-05-05 12:34:56.1817,1909-12-25 -1909-05-05 12:34:56.1818,1909-12-25 -1909-05-05 12:34:56.1819,1909-12-25 -1909-05-05 12:34:56.182,1909-12-25 -1909-05-05 12:34:56.1821,1909-12-25 -1909-05-05 12:34:56.1822,1909-12-25 -1909-05-05 12:34:56.1823,1909-12-25 -1909-05-05 12:34:56.1824,1909-12-25 -1909-05-05 12:34:56.1825,1909-12-25 -1909-05-05 12:34:56.1826,1909-12-25 -1909-05-05 12:34:56.1827,1909-12-25 -1909-05-05 12:34:56.1828,1909-12-25 -1909-05-05 12:34:56.1829,1909-12-25 -1909-05-05 12:34:56.183,1909-12-25 -1909-05-05 12:34:56.1831,1909-12-25 -1909-05-05 12:34:56.1832,1909-12-25 -1909-05-05 12:34:56.1833,1909-12-25 -1909-05-05 12:34:56.1834,1909-12-25 -1909-05-05 12:34:56.1835,1909-12-25 -1909-05-05 12:34:56.1836,1909-12-25 -1909-05-05 12:34:56.1837,1909-12-25 -1909-05-05 12:34:56.1838,1909-12-25 -1909-05-05 12:34:56.1839,1909-12-25 -1909-05-05 12:34:56.184,1909-12-25 -1909-05-05 12:34:56.1841,1909-12-25 -1909-05-05 12:34:56.1842,1909-12-25 -1909-05-05 12:34:56.1843,1909-12-25 -1909-05-05 12:34:56.1844,1909-12-25 -1909-05-05 12:34:56.1845,1909-12-25 -1909-05-05 12:34:56.1846,1909-12-25 -1909-05-05 12:34:56.1847,1909-12-25 -1909-05-05 12:34:56.1848,1909-12-25 -1909-05-05 12:34:56.1849,1909-12-25 -1909-05-05 12:34:56.185,1909-12-25 -1909-05-05 12:34:56.1851,1909-12-25 -1909-05-05 12:34:56.1852,1909-12-25 -1909-05-05 12:34:56.1853,1909-12-25 -1909-05-05 12:34:56.1854,1909-12-25 -1909-05-05 12:34:56.1855,1909-12-25 -1909-05-05 12:34:56.1856,1909-12-25 -1909-05-05 12:34:56.1857,1909-12-25 -1909-05-05 12:34:56.1858,1909-12-25 -1909-05-05 12:34:56.1859,1909-12-25 -1909-05-05 12:34:56.186,1909-12-25 -1909-05-05 12:34:56.1861,1909-12-25 -1909-05-05 12:34:56.1862,1909-12-25 -1909-05-05 12:34:56.1863,1909-12-25 -1909-05-05 12:34:56.1864,1909-12-25 -1909-05-05 12:34:56.1865,1909-12-25 -1909-05-05 12:34:56.1866,1909-12-25 -1909-05-05 12:34:56.1867,1909-12-25 -1909-05-05 12:34:56.1868,1909-12-25 -1909-05-05 12:34:56.1869,1909-12-25 -1909-05-05 12:34:56.187,1909-12-25 -1909-05-05 12:34:56.1871,1909-12-25 -1909-05-05 12:34:56.1872,1909-12-25 -1909-05-05 12:34:56.1873,1909-12-25 -1909-05-05 12:34:56.1874,1909-12-25 -1909-05-05 12:34:56.1875,1909-12-25 -1909-05-05 12:34:56.1876,1909-12-25 -1909-05-05 12:34:56.1877,1909-12-25 -1909-05-05 12:34:56.1878,1909-12-25 -1909-05-05 12:34:56.1879,1909-12-25 -1909-05-05 12:34:56.188,1909-12-25 -1909-05-05 12:34:56.1881,1909-12-25 -1909-05-05 12:34:56.1882,1909-12-25 -1909-05-05 12:34:56.1883,1909-12-25 -1909-05-05 12:34:56.1884,1909-12-25 -1909-05-05 12:34:56.1885,1909-12-25 -1909-05-05 12:34:56.1886,1909-12-25 -1909-05-05 12:34:56.1887,1909-12-25 -1909-05-05 12:34:56.1888,1909-12-25 -1909-05-05 12:34:56.1889,1909-12-25 -1909-05-05 12:34:56.189,1909-12-25 -1909-05-05 12:34:56.1891,1909-12-25 -1909-05-05 12:34:56.1892,1909-12-25 -1909-05-05 12:34:56.1893,1909-12-25 -1909-05-05 12:34:56.1894,1909-12-25 -1909-05-05 12:34:56.1895,1909-12-25 -1909-05-05 12:34:56.1896,1909-12-25 -1909-05-05 12:34:56.1897,1909-12-25 -1909-05-05 12:34:56.1898,1909-12-25 -1909-05-05 12:34:56.1899,1909-12-25 -1909-05-05 12:34:56.19,1909-12-25 -1909-05-05 12:34:56.1901,1909-12-25 -1909-05-05 12:34:56.1902,1909-12-25 -1909-05-05 12:34:56.1903,1909-12-25 -1909-05-05 12:34:56.1904,1909-12-25 -1909-05-05 12:34:56.1905,1909-12-25 -1909-05-05 12:34:56.1906,1909-12-25 -1909-05-05 12:34:56.1907,1909-12-25 -1909-05-05 12:34:56.1908,1909-12-25 -1909-05-05 12:34:56.1909,1909-12-25 -1909-05-05 12:34:56.191,1909-12-25 -1909-05-05 12:34:56.1911,1909-12-25 -1909-05-05 12:34:56.1912,1909-12-25 -1909-05-05 12:34:56.1913,1909-12-25 -1909-05-05 12:34:56.1914,1909-12-25 -1909-05-05 12:34:56.1915,1909-12-25 -1909-05-05 12:34:56.1916,1909-12-25 -1909-05-05 12:34:56.1917,1909-12-25 -1909-05-05 12:34:56.1918,1909-12-25 -1909-05-05 12:34:56.1919,1909-12-25 -1909-05-05 12:34:56.192,1909-12-25 -1909-05-05 12:34:56.1921,1909-12-25 -1909-05-05 12:34:56.1922,1909-12-25 -1909-05-05 12:34:56.1923,1909-12-25 -1909-05-05 12:34:56.1924,1909-12-25 -1909-05-05 12:34:56.1925,1909-12-25 -1909-05-05 12:34:56.1926,1909-12-25 -1909-05-05 12:34:56.1927,1909-12-25 -1909-05-05 12:34:56.1928,1909-12-25 -1909-05-05 12:34:56.1929,1909-12-25 -1909-05-05 12:34:56.193,1909-12-25 -1909-05-05 12:34:56.1931,1909-12-25 -1909-05-05 12:34:56.1932,1909-12-25 -1909-05-05 12:34:56.1933,1909-12-25 -1909-05-05 12:34:56.1934,1909-12-25 -1909-05-05 12:34:56.1935,1909-12-25 -1909-05-05 12:34:56.1936,1909-12-25 -1909-05-05 12:34:56.1937,1909-12-25 -1909-05-05 12:34:56.1938,1909-12-25 -1909-05-05 12:34:56.1939,1909-12-25 -1909-05-05 12:34:56.194,1909-12-25 -1909-05-05 12:34:56.1941,1909-12-25 -1909-05-05 12:34:56.1942,1909-12-25 -1909-05-05 12:34:56.1943,1909-12-25 -1909-05-05 12:34:56.1944,1909-12-25 -1909-05-05 12:34:56.1945,1909-12-25 -1909-05-05 12:34:56.1946,1909-12-25 -1909-05-05 12:34:56.1947,1909-12-25 -1909-05-05 12:34:56.1948,1909-12-25 -1909-05-05 12:34:56.1949,1909-12-25 -1909-05-05 12:34:56.195,1909-12-25 -1909-05-05 12:34:56.1951,1909-12-25 -1909-05-05 12:34:56.1952,1909-12-25 -1909-05-05 12:34:56.1953,1909-12-25 -1909-05-05 12:34:56.1954,1909-12-25 -1909-05-05 12:34:56.1955,1909-12-25 -1909-05-05 12:34:56.1956,1909-12-25 -1909-05-05 12:34:56.1957,1909-12-25 -1909-05-05 12:34:56.1958,1909-12-25 -1909-05-05 12:34:56.1959,1909-12-25 -1909-05-05 12:34:56.196,1909-12-25 -1909-05-05 12:34:56.1961,1909-12-25 -1909-05-05 12:34:56.1962,1909-12-25 -1909-05-05 12:34:56.1963,1909-12-25 -1909-05-05 12:34:56.1964,1909-12-25 -1909-05-05 12:34:56.1965,1909-12-25 -1909-05-05 12:34:56.1966,1909-12-25 -1909-05-05 12:34:56.1967,1909-12-25 -1909-05-05 12:34:56.1968,1909-12-25 -1909-05-05 12:34:56.1969,1909-12-25 -1909-05-05 12:34:56.197,1909-12-25 -1909-05-05 12:34:56.1971,1909-12-25 -1909-05-05 12:34:56.1972,1909-12-25 -1909-05-05 12:34:56.1973,1909-12-25 -1909-05-05 12:34:56.1974,1909-12-25 -1909-05-05 12:34:56.1975,1909-12-25 -1909-05-05 12:34:56.1976,1909-12-25 -1909-05-05 12:34:56.1977,1909-12-25 -1909-05-05 12:34:56.1978,1909-12-25 -1909-05-05 12:34:56.1979,1909-12-25 -1909-05-05 12:34:56.198,1909-12-25 -1909-05-05 12:34:56.1981,1909-12-25 -1909-05-05 12:34:56.1982,1909-12-25 -1909-05-05 12:34:56.1983,1909-12-25 -1909-05-05 12:34:56.1984,1909-12-25 -1909-05-05 12:34:56.1985,1909-12-25 -1909-05-05 12:34:56.1986,1909-12-25 -1909-05-05 12:34:56.1987,1909-12-25 -1909-05-05 12:34:56.1988,1909-12-25 -1909-05-05 12:34:56.1989,1909-12-25 -1909-05-05 12:34:56.199,1909-12-25 -1909-05-05 12:34:56.1991,1909-12-25 -1909-05-05 12:34:56.1992,1909-12-25 -1909-05-05 12:34:56.1993,1909-12-25 -1909-05-05 12:34:56.1994,1909-12-25 -1909-05-05 12:34:56.1995,1909-12-25 -1909-05-05 12:34:56.1996,1909-12-25 -1909-05-05 12:34:56.1997,1909-12-25 -1909-05-05 12:34:56.1998,1909-12-25 -1909-05-05 12:34:56.1999,1909-12-25 diff --git a/flink-connectors/flink-orc/src/test/resources/decimal.dat b/flink-connectors/flink-orc/src/test/resources/decimal.dat deleted file mode 100644 index 42010b518c91e..0000000000000 --- a/flink-connectors/flink-orc/src/test/resources/decimal.dat +++ /dev/null @@ -1,6000 +0,0 @@ --1000.5 --999.6 --998.7 --997.8 --996.9 --995.1 --994.11 --993.12 --992.13 --991.14 --990.15 --989.16 --988.17 --987.18 --986.19 --985.2 --984.21 --983.22 --982.23 --981.24 --980.25 --979.26 --978.27 --977.28 --976.29 --975.3 --974.31 --973.32 --972.33 --971.34 --970.35 --969.36 --968.37 --967.38 --966.39 --965.4 --964.41 --963.42 --962.43 --961.44 --960.45 --959.46 --958.47 --957.48 --956.49 --955.5 --954.51 --953.52 --952.53 --951.54 --950.55 --949.56 --948.57 --947.58 --946.59 --945.6 --944.61 --943.62 --942.63 --941.64 --940.65 --939.66 --938.67 --937.68 --936.69 --935.7 --934.71 --933.72 --932.73 --931.74 --930.75 --929.76 --928.77 --927.78 --926.79 --925.8 --924.81 --923.82 --922.83 --921.84 --920.85 --919.86 --918.87 --917.88 --916.89 --915.9 --914.91 --913.92 --912.93 --911.94 --910.95 --909.96 --908.97 --907.98 --906.99 --905.1 --904.101 --903.102 --902.103 --901.104 --900.105 --899.106 --898.107 --897.108 --896.109 --895.11 --894.111 --893.112 --892.113 --891.114 --890.115 --889.116 --888.117 --887.118 --886.119 --885.12 --884.121 --883.122 --882.123 --881.124 --880.125 --879.126 --878.127 --877.128 --876.129 --875.13 --874.131 --873.132 --872.133 --871.134 --870.135 --869.136 --868.137 --867.138 --866.139 --865.14 --864.141 --863.142 --862.143 --861.144 --860.145 --859.146 --858.147 --857.148 --856.149 --855.15 --854.151 --853.152 --852.153 --851.154 --850.155 --849.156 --848.157 --847.158 --846.159 --845.16 --844.161 --843.162 --842.163 --841.164 --840.165 --839.166 --838.167 --837.168 --836.169 --835.17 --834.171 --833.172 --832.173 --831.174 --830.175 --829.176 --828.177 --827.178 --826.179 --825.18 --824.181 --823.182 --822.183 --821.184 --820.185 --819.186 --818.187 --817.188 --816.189 --815.19 --814.191 --813.192 --812.193 --811.194 --810.195 --809.196 --808.197 --807.198 --806.199 --805.2 --804.201 --803.202 --802.203 --801.204 --800.205 --799.206 --798.207 --797.208 --796.209 --795.21 --794.211 --793.212 --792.213 --791.214 --790.215 --789.216 --788.217 --787.218 --786.219 --785.22 --784.221 --783.222 --782.223 --781.224 --780.225 --779.226 --778.227 --777.228 --776.229 --775.23 --774.231 --773.232 --772.233 --771.234 --770.235 --769.236 --768.237 --767.238 --766.239 --765.24 --764.241 --763.242 --762.243 --761.244 --760.245 --759.246 --758.247 --757.248 --756.249 --755.25 --754.251 --753.252 --752.253 --751.254 --750.255 --749.256 --748.257 --747.258 --746.259 --745.26 --744.261 --743.262 --742.263 --741.264 --740.265 --739.266 --738.267 --737.268 --736.269 --735.27 --734.271 --733.272 --732.273 --731.274 --730.275 --729.276 --728.277 --727.278 --726.279 --725.28 --724.281 --723.282 --722.283 --721.284 --720.285 --719.286 --718.287 --717.288 --716.289 --715.29 --714.291 --713.292 --712.293 --711.294 --710.295 --709.296 --708.297 --707.298 --706.299 --705.3 --704.301 --703.302 --702.303 --701.304 --700.305 --699.306 --698.307 --697.308 --696.309 --695.31 --694.311 --693.312 --692.313 --691.314 --690.315 --689.316 --688.317 --687.318 --686.319 --685.32 --684.321 --683.322 --682.323 --681.324 --680.325 --679.326 --678.327 --677.328 --676.329 --675.33 --674.331 --673.332 --672.333 --671.334 --670.335 --669.336 --668.337 --667.338 --666.339 --665.34 --664.341 --663.342 --662.343 --661.344 --660.345 --659.346 --658.347 --657.348 --656.349 --655.35 --654.351 --653.352 --652.353 --651.354 --650.355 --649.356 --648.357 --647.358 --646.359 --645.36 --644.361 --643.362 --642.363 --641.364 --640.365 --639.366 --638.367 --637.368 --636.369 --635.37 --634.371 --633.372 --632.373 --631.374 --630.375 --629.376 --628.377 --627.378 --626.379 --625.38 --624.381 --623.382 --622.383 --621.384 --620.385 --619.386 --618.387 --617.388 --616.389 --615.39 --614.391 --613.392 --612.393 --611.394 --610.395 --609.396 --608.397 --607.398 --606.399 --605.4 --604.401 --603.402 --602.403 --601.404 --600.405 --599.406 --598.407 --597.408 --596.409 --595.41 --594.411 --593.412 --592.413 --591.414 --590.415 --589.416 --588.417 --587.418 --586.419 --585.42 --584.421 --583.422 --582.423 --581.424 --580.425 --579.426 --578.427 --577.428 --576.429 --575.43 --574.431 --573.432 --572.433 --571.434 --570.435 --569.436 --568.437 --567.438 --566.439 --565.44 --564.441 --563.442 --562.443 --561.444 --560.445 --559.446 --558.447 --557.448 --556.449 --555.45 --554.451 --553.452 --552.453 --551.454 --550.455 --549.456 --548.457 --547.458 --546.459 --545.46 --544.461 --543.462 --542.463 --541.464 --540.465 --539.466 --538.467 --537.468 --536.469 --535.47 --534.471 --533.472 --532.473 --531.474 --530.475 --529.476 --528.477 --527.478 --526.479 --525.48 --524.481 --523.482 --522.483 --521.484 --520.485 --519.486 --518.487 --517.488 --516.489 --515.49 --514.491 --513.492 --512.493 --511.494 --510.495 --509.496 --508.497 --507.498 --506.499 --505.5 --504.501 --503.502 --502.503 --501.504 --500.505 --499.506 --498.507 --497.508 --496.509 --495.51 --494.511 --493.512 --492.513 --491.514 --490.515 --489.516 --488.517 --487.518 --486.519 --485.52 --484.521 --483.522 --482.523 --481.524 --480.525 --479.526 --478.527 --477.528 --476.529 --475.53 --474.531 --473.532 --472.533 --471.534 --470.535 --469.536 --468.537 --467.538 --466.539 --465.54 --464.541 --463.542 --462.543 --461.544 --460.545 --459.546 --458.547 --457.548 --456.549 --455.55 --454.551 --453.552 --452.553 --451.554 --450.555 --449.556 --448.557 --447.558 --446.559 --445.56 --444.561 --443.562 --442.563 --441.564 --440.565 --439.566 --438.567 --437.568 --436.569 --435.57 --434.571 --433.572 --432.573 --431.574 --430.575 --429.576 --428.577 --427.578 --426.579 --425.58 --424.581 --423.582 --422.583 --421.584 --420.585 --419.586 --418.587 --417.588 --416.589 --415.59 --414.591 --413.592 --412.593 --411.594 --410.595 --409.596 --408.597 --407.598 --406.599 --405.6 --404.601 --403.602 --402.603 --401.604 --400.605 --399.606 --398.607 --397.608 --396.609 --395.61 --394.611 --393.612 --392.613 --391.614 --390.615 --389.616 --388.617 --387.618 --386.619 --385.62 --384.621 --383.622 --382.623 --381.624 --380.625 --379.626 --378.627 --377.628 --376.629 --375.63 --374.631 --373.632 --372.633 --371.634 --370.635 --369.636 --368.637 --367.638 --366.639 --365.64 --364.641 --363.642 --362.643 --361.644 --360.645 --359.646 --358.647 --357.648 --356.649 --355.65 --354.651 --353.652 --352.653 --351.654 --350.655 --349.656 --348.657 --347.658 --346.659 --345.66 --344.661 --343.662 --342.663 --341.664 --340.665 --339.666 --338.667 --337.668 --336.669 --335.67 --334.671 --333.672 --332.673 --331.674 --330.675 --329.676 --328.677 --327.678 --326.679 --325.68 --324.681 --323.682 --322.683 --321.684 --320.685 --319.686 --318.687 --317.688 --316.689 --315.69 --314.691 --313.692 --312.693 --311.694 --310.695 --309.696 --308.697 --307.698 --306.699 --305.7 --304.701 --303.702 --302.703 --301.704 --300.705 --299.706 --298.707 --297.708 --296.709 --295.71 --294.711 --293.712 --292.713 --291.714 --290.715 --289.716 --288.717 --287.718 --286.719 --285.72 --284.721 --283.722 --282.723 --281.724 --280.725 --279.726 --278.727 --277.728 --276.729 --275.73 --274.731 --273.732 --272.733 --271.734 --270.735 --269.736 --268.737 --267.738 --266.739 --265.74 --264.741 --263.742 --262.743 --261.744 --260.745 --259.746 --258.747 --257.748 --256.749 --255.75 --254.751 --253.752 --252.753 --251.754 --250.755 --249.756 --248.757 --247.758 --246.759 --245.76 --244.761 --243.762 --242.763 --241.764 --240.765 --239.766 --238.767 --237.768 --236.769 --235.77 --234.771 --233.772 --232.773 --231.774 --230.775 --229.776 --228.777 --227.778 --226.779 --225.78 --224.781 --223.782 --222.783 --221.784 --220.785 --219.786 --218.787 --217.788 --216.789 --215.79 --214.791 --213.792 --212.793 --211.794 --210.795 --209.796 --208.797 --207.798 --206.799 --205.8 --204.801 --203.802 --202.803 --201.804 --200.805 --199.806 --198.807 --197.808 --196.809 --195.81 --194.811 --193.812 --192.813 --191.814 --190.815 --189.816 --188.817 --187.818 --186.819 --185.82 --184.821 --183.822 --182.823 --181.824 --180.825 --179.826 --178.827 --177.828 --176.829 --175.83 --174.831 --173.832 --172.833 --171.834 --170.835 --169.836 --168.837 --167.838 --166.839 --165.84 --164.841 --163.842 --162.843 --161.844 --160.845 --159.846 --158.847 --157.848 --156.849 --155.85 --154.851 --153.852 --152.853 --151.854 --150.855 --149.856 --148.857 --147.858 --146.859 --145.86 --144.861 --143.862 --142.863 --141.864 --140.865 --139.866 --138.867 --137.868 --136.869 --135.87 --134.871 --133.872 --132.873 --131.874 --130.875 --129.876 --128.877 --127.878 --126.879 --125.88 --124.881 --123.882 --122.883 --121.884 --120.885 --119.886 --118.887 --117.888 --116.889 --115.89 --114.891 --113.892 --112.893 --111.894 --110.895 --109.896 --108.897 --107.898 --106.899 --105.9 --104.901 --103.902 --102.903 --101.904 --100.905 --99.906 --98.907 --97.908 --96.909 --95.91 --94.911 --93.912 --92.913 --91.914 --90.915 --89.916 --88.917 --87.918 --86.919 --85.92 --84.921 --83.922 --82.923 --81.924 --80.925 --79.926 --78.927 --77.928 --76.929 --75.93 --74.931 --73.932 --72.933 --71.934 --70.935 --69.936 --68.937 --67.938 --66.939 --65.94 --64.941 --63.942 --62.943 --61.944 --60.945 --59.946 --58.947 --57.948 --56.949 --55.95 --54.951 --53.952 --52.953 --51.954 --50.955 --49.956 --48.957 --47.958 --46.959 --45.96 --44.961 --43.962 --42.963 --41.964 --40.965 --39.966 --38.967 --37.968 --36.969 --35.97 --34.971 --33.972 --32.973 --31.974 --30.975 --29.976 --28.977 --27.978 --26.979 --25.98 --24.981 --23.982 --22.983 --21.984 --20.985 --19.986 --18.987 --17.988 --16.989 --15.99 --14.991 --13.992 --12.993 --11.994 --10.995 --9.996 --8.997 --7.998 --6.999 --5.1 --4.1001 --3.1002 --2.1003 --1.1004 -0.1005 -1.1006 -2.1007 -3.1008 -4.1009 -5.101 -6.1011 -7.1012 -8.1013 -9.1014 -10.1015 -11.1016 -12.1017 -13.1018 -14.1019 -15.102 -16.1021 -17.1022 -18.1023 -19.1024 -20.1025 -21.1026 -22.1027 -23.1028 -24.1029 -25.103 -26.1031 -27.1032 -28.1033 -29.1034 -30.1035 -31.1036 -32.1037 -33.1038 -34.1039 -35.104 -36.1041 -37.1042 -38.1043 -39.1044 -40.1045 -41.1046 -42.1047 -43.1048 -44.1049 -45.105 -46.1051 -47.1052 -48.1053 -49.1054 -50.1055 -51.1056 -52.1057 -53.1058 -54.1059 -55.106 -56.1061 -57.1062 -58.1063 -59.1064 -60.1065 -61.1066 -62.1067 -63.1068 -64.1069 -65.107 -66.1071 -67.1072 -68.1073 -69.1074 -70.1075 -71.1076 -72.1077 -73.1078 -74.1079 -75.108 -76.1081 -77.1082 -78.1083 -79.1084 -80.1085 -81.1086 -82.1087 -83.1088 -84.1089 -85.109 -86.1091 -87.1092 -88.1093 -89.1094 -90.1095 -91.1096 -92.1097 -93.1098 -94.1099 -95.11 -96.1101 -97.1102 -98.1103 -99.1104 -100.1105 -101.1106 -102.1107 -103.1108 -104.1109 -105.111 -106.1111 -107.1112 -108.1113 -109.1114 -110.1115 -111.1116 -112.1117 -113.1118 -114.1119 -115.112 -116.1121 -117.1122 -118.1123 -119.1124 -120.1125 -121.1126 -122.1127 -123.1128 -124.1129 -125.113 -126.1131 -127.1132 -128.1133 -129.1134 -130.1135 -131.1136 -132.1137 -133.1138 -134.1139 -135.114 -136.1141 -137.1142 -138.1143 -139.1144 -140.1145 -141.1146 -142.1147 -143.1148 -144.1149 -145.115 -146.1151 -147.1152 -148.1153 -149.1154 -150.1155 -151.1156 -152.1157 -153.1158 -154.1159 -155.116 -156.1161 -157.1162 -158.1163 -159.1164 -160.1165 -161.1166 -162.1167 -163.1168 -164.1169 -165.117 -166.1171 -167.1172 -168.1173 -169.1174 -170.1175 -171.1176 -172.1177 -173.1178 -174.1179 -175.118 -176.1181 -177.1182 -178.1183 -179.1184 -180.1185 -181.1186 -182.1187 -183.1188 -184.1189 -185.119 -186.1191 -187.1192 -188.1193 -189.1194 -190.1195 -191.1196 -192.1197 -193.1198 -194.1199 -195.12 -196.1201 -197.1202 -198.1203 -199.1204 -200.1205 -201.1206 -202.1207 -203.1208 -204.1209 -205.121 -206.1211 -207.1212 -208.1213 -209.1214 -210.1215 -211.1216 -212.1217 -213.1218 -214.1219 -215.122 -216.1221 -217.1222 -218.1223 -219.1224 -220.1225 -221.1226 -222.1227 -223.1228 -224.1229 -225.123 -226.1231 -227.1232 -228.1233 -229.1234 -230.1235 -231.1236 -232.1237 -233.1238 -234.1239 -235.124 -236.1241 -237.1242 -238.1243 -239.1244 -240.1245 -241.1246 -242.1247 -243.1248 -244.1249 -245.125 -246.1251 -247.1252 -248.1253 -249.1254 -250.1255 -251.1256 -252.1257 -253.1258 -254.1259 -255.126 -256.1261 -257.1262 -258.1263 -259.1264 -260.1265 -261.1266 -262.1267 -263.1268 -264.1269 -265.127 -266.1271 -267.1272 -268.1273 -269.1274 -270.1275 -271.1276 -272.1277 -273.1278 -274.1279 -275.128 -276.1281 -277.1282 -278.1283 -279.1284 -280.1285 -281.1286 -282.1287 -283.1288 -284.1289 -285.129 -286.1291 -287.1292 -288.1293 -289.1294 -290.1295 -291.1296 -292.1297 -293.1298 -294.1299 -295.13 -296.1301 -297.1302 -298.1303 -299.1304 -300.1305 -301.1306 -302.1307 -303.1308 -304.1309 -305.131 -306.1311 -307.1312 -308.1313 -309.1314 -310.1315 -311.1316 -312.1317 -313.1318 -314.1319 -315.132 -316.1321 -317.1322 -318.1323 -319.1324 -320.1325 -321.1326 -322.1327 -323.1328 -324.1329 -325.133 -326.1331 -327.1332 -328.1333 -329.1334 -330.1335 -331.1336 -332.1337 -333.1338 -334.1339 -335.134 -336.1341 -337.1342 -338.1343 -339.1344 -340.1345 -341.1346 -342.1347 -343.1348 -344.1349 -345.135 -346.1351 -347.1352 -348.1353 -349.1354 -350.1355 -351.1356 -352.1357 -353.1358 -354.1359 -355.136 -356.1361 -357.1362 -358.1363 -359.1364 -360.1365 -361.1366 -362.1367 -363.1368 -364.1369 -365.137 -366.1371 -367.1372 -368.1373 -369.1374 -370.1375 -371.1376 -372.1377 -373.1378 -374.1379 -375.138 -376.1381 -377.1382 -378.1383 -379.1384 -380.1385 -381.1386 -382.1387 -383.1388 -384.1389 -385.139 -386.1391 -387.1392 -388.1393 -389.1394 -390.1395 -391.1396 -392.1397 -393.1398 -394.1399 -395.14 -396.1401 -397.1402 -398.1403 -399.1404 -400.1405 -401.1406 -402.1407 -403.1408 -404.1409 -405.141 -406.1411 -407.1412 -408.1413 -409.1414 -410.1415 -411.1416 -412.1417 -413.1418 -414.1419 -415.142 -416.1421 -417.1422 -418.1423 -419.1424 -420.1425 -421.1426 -422.1427 -423.1428 -424.1429 -425.143 -426.1431 -427.1432 -428.1433 -429.1434 -430.1435 -431.1436 -432.1437 -433.1438 -434.1439 -435.144 -436.1441 -437.1442 -438.1443 -439.1444 -440.1445 -441.1446 -442.1447 -443.1448 -444.1449 -445.145 -446.1451 -447.1452 -448.1453 -449.1454 -450.1455 -451.1456 -452.1457 -453.1458 -454.1459 -455.146 -456.1461 -457.1462 -458.1463 -459.1464 -460.1465 -461.1466 -462.1467 -463.1468 -464.1469 -465.147 -466.1471 -467.1472 -468.1473 -469.1474 -470.1475 -471.1476 -472.1477 -473.1478 -474.1479 -475.148 -476.1481 -477.1482 -478.1483 -479.1484 -480.1485 -481.1486 -482.1487 -483.1488 -484.1489 -485.149 -486.1491 -487.1492 -488.1493 -489.1494 -490.1495 -491.1496 -492.1497 -493.1498 -494.1499 -495.15 -496.1501 -497.1502 -498.1503 -499.1504 -500.1505 -501.1506 -502.1507 -503.1508 -504.1509 -505.151 -506.1511 -507.1512 -508.1513 -509.1514 -510.1515 -511.1516 -512.1517 -513.1518 -514.1519 -515.152 -516.1521 -517.1522 -518.1523 -519.1524 -520.1525 -521.1526 -522.1527 -523.1528 -524.1529 -525.153 -526.1531 -527.1532 -528.1533 -529.1534 -530.1535 -531.1536 -532.1537 -533.1538 -534.1539 -535.154 -536.1541 -537.1542 -538.1543 -539.1544 -540.1545 -541.1546 -542.1547 -543.1548 -544.1549 -545.155 -546.1551 -547.1552 -548.1553 -549.1554 -550.1555 -551.1556 -552.1557 -553.1558 -554.1559 -555.156 -556.1561 -557.1562 -558.1563 -559.1564 -560.1565 -561.1566 -562.1567 -563.1568 -564.1569 -565.157 -566.1571 -567.1572 -568.1573 -569.1574 -570.1575 -571.1576 -572.1577 -573.1578 -574.1579 -575.158 -576.1581 -577.1582 -578.1583 -579.1584 -580.1585 -581.1586 -582.1587 -583.1588 -584.1589 -585.159 -586.1591 -587.1592 -588.1593 -589.1594 -590.1595 -591.1596 -592.1597 -593.1598 -594.1599 -595.16 -596.1601 -597.1602 -598.1603 -599.1604 -600.1605 -601.1606 -602.1607 -603.1608 -604.1609 -605.161 -606.1611 -607.1612 -608.1613 -609.1614 -610.1615 -611.1616 -612.1617 -613.1618 -614.1619 -615.162 -616.1621 -617.1622 -618.1623 -619.1624 -620.1625 -621.1626 -622.1627 -623.1628 -624.1629 -625.163 -626.1631 -627.1632 -628.1633 -629.1634 -630.1635 -631.1636 -632.1637 -633.1638 -634.1639 -635.164 -636.1641 -637.1642 -638.1643 -639.1644 -640.1645 -641.1646 -642.1647 -643.1648 -644.1649 -645.165 -646.1651 -647.1652 -648.1653 -649.1654 -650.1655 -651.1656 -652.1657 -653.1658 -654.1659 -655.166 -656.1661 -657.1662 -658.1663 -659.1664 -660.1665 -661.1666 -662.1667 -663.1668 -664.1669 -665.167 -666.1671 -667.1672 -668.1673 -669.1674 -670.1675 -671.1676 -672.1677 -673.1678 -674.1679 -675.168 -676.1681 -677.1682 -678.1683 -679.1684 -680.1685 -681.1686 -682.1687 -683.1688 -684.1689 -685.169 -686.1691 -687.1692 -688.1693 -689.1694 -690.1695 -691.1696 -692.1697 -693.1698 -694.1699 -695.17 -696.1701 -697.1702 -698.1703 -699.1704 -700.1705 -701.1706 -702.1707 -703.1708 -704.1709 -705.171 -706.1711 -707.1712 -708.1713 -709.1714 -710.1715 -711.1716 -712.1717 -713.1718 -714.1719 -715.172 -716.1721 -717.1722 -718.1723 -719.1724 -720.1725 -721.1726 -722.1727 -723.1728 -724.1729 -725.173 -726.1731 -727.1732 -728.1733 -729.1734 -730.1735 -731.1736 -732.1737 -733.1738 -734.1739 -735.174 -736.1741 -737.1742 -738.1743 -739.1744 -740.1745 -741.1746 -742.1747 -743.1748 -744.1749 -745.175 -746.1751 -747.1752 -748.1753 -749.1754 -750.1755 -751.1756 -752.1757 -753.1758 -754.1759 -755.176 -756.1761 -757.1762 -758.1763 -759.1764 -760.1765 -761.1766 -762.1767 -763.1768 -764.1769 -765.177 -766.1771 -767.1772 -768.1773 -769.1774 -770.1775 -771.1776 -772.1777 -773.1778 -774.1779 -775.178 -776.1781 -777.1782 -778.1783 -779.1784 -780.1785 -781.1786 -782.1787 -783.1788 -784.1789 -785.179 -786.1791 -787.1792 -788.1793 -789.1794 -790.1795 -791.1796 -792.1797 -793.1798 -794.1799 -795.18 -796.1801 -797.1802 -798.1803 -799.1804 -800.1805 -801.1806 -802.1807 -803.1808 -804.1809 -805.181 -806.1811 -807.1812 -808.1813 -809.1814 -810.1815 -811.1816 -812.1817 -813.1818 -814.1819 -815.182 -816.1821 -817.1822 -818.1823 -819.1824 -820.1825 -821.1826 -822.1827 -823.1828 -824.1829 -825.183 -826.1831 -827.1832 -828.1833 -829.1834 -830.1835 -831.1836 -832.1837 -833.1838 -834.1839 -835.184 -836.1841 -837.1842 -838.1843 -839.1844 -840.1845 -841.1846 -842.1847 -843.1848 -844.1849 -845.185 -846.1851 -847.1852 -848.1853 -849.1854 -850.1855 -851.1856 -852.1857 -853.1858 -854.1859 -855.186 -856.1861 -857.1862 -858.1863 -859.1864 -860.1865 -861.1866 -862.1867 -863.1868 -864.1869 -865.187 -866.1871 -867.1872 -868.1873 -869.1874 -870.1875 -871.1876 -872.1877 -873.1878 -874.1879 -875.188 -876.1881 -877.1882 -878.1883 -879.1884 -880.1885 -881.1886 -882.1887 -883.1888 -884.1889 -885.189 -886.1891 -887.1892 -888.1893 -889.1894 -890.1895 -891.1896 -892.1897 -893.1898 -894.1899 -895.19 -896.1901 -897.1902 -898.1903 -899.1904 -900.1905 -901.1906 -902.1907 -903.1908 -904.1909 -905.191 -906.1911 -907.1912 -908.1913 -909.1914 -910.1915 -911.1916 -912.1917 -913.1918 -914.1919 -915.192 -916.1921 -917.1922 -918.1923 -919.1924 -920.1925 -921.1926 -922.1927 -923.1928 -924.1929 -925.193 -926.1931 -927.1932 -928.1933 -929.1934 -930.1935 -931.1936 -932.1937 -933.1938 -934.1939 -935.194 -936.1941 -937.1942 -938.1943 -939.1944 -940.1945 -941.1946 -942.1947 -943.1948 -944.1949 -945.195 -946.1951 -947.1952 -948.1953 -949.1954 -950.1955 -951.1956 -952.1957 -953.1958 -954.1959 -955.196 -956.1961 -957.1962 -958.1963 -959.1964 -960.1965 -961.1966 -962.1967 -963.1968 -964.1969 -965.197 -966.1971 -967.1972 -968.1973 -969.1974 -970.1975 -971.1976 -972.1977 -973.1978 -974.1979 -975.198 -976.1981 -977.1982 -978.1983 -979.1984 -980.1985 -981.1986 -982.1987 -983.1988 -984.1989 -985.199 -986.1991 -987.1992 -988.1993 -989.1994 -990.1995 -991.1996 -992.1997 -993.1998 -994.1999 -995.2 -996.2001 -997.2002 -998.2003 -999.2004 -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -null -0.1 -1.2 -2.3 -3.4 -4.5 -5.6 -6.7 -7.8 -8.9 -9.1 -10.11 -11.12 -12.13 -13.14 -14.15 -15.16 -16.17 -17.18 -18.19 -19.2 -20.21 -21.22 -22.23 -23.24 -24.25 -25.26 -26.27 -27.28 -28.29 -29.3 -30.31 -31.32 -32.33 -33.34 -34.35 -35.36 -36.37 -37.38 -38.39 -39.4 -40.41 -41.42 -42.43 -43.44 -44.45 -45.46 -46.47 -47.48 -48.49 -49.5 -50.51 -51.52 -52.53 -53.54 -54.55 -55.56 -56.57 -57.58 -58.59 -59.6 -60.61 -61.62 -62.63 -63.64 -64.65 -65.66 -66.67 -67.68 -68.69 -69.7 -70.71 -71.72 -72.73 -73.74 -74.75 -75.76 -76.77 -77.78 -78.79 -79.8 -80.81 -81.82 -82.83 -83.84 -84.85 -85.86 -86.87 -87.88 -88.89 -89.9 -90.91 -91.92 -92.93 -93.94 -94.95 -95.96 -96.97 -97.98 -98.99 -99.1 -100.101 -101.102 -102.103 -103.104 -104.105 -105.106 -106.107 -107.108 -108.109 -109.11 -110.111 -111.112 -112.113 -113.114 -114.115 -115.116 -116.117 -117.118 -118.119 -119.12 -120.121 -121.122 -122.123 -123.124 -124.125 -125.126 -126.127 -127.128 -128.129 -129.13 -130.131 -131.132 -132.133 -133.134 -134.135 -135.136 -136.137 -137.138 -138.139 -139.14 -140.141 -141.142 -142.143 -143.144 -144.145 -145.146 -146.147 -147.148 -148.149 -149.15 -150.151 -151.152 -152.153 -153.154 -154.155 -155.156 -156.157 -157.158 -158.159 -159.16 -160.161 -161.162 -162.163 -163.164 -164.165 -165.166 -166.167 -167.168 -168.169 -169.17 -170.171 -171.172 -172.173 -173.174 -174.175 -175.176 -176.177 -177.178 -178.179 -179.18 -180.181 -181.182 -182.183 -183.184 -184.185 -185.186 -186.187 -187.188 -188.189 -189.19 -190.191 -191.192 -192.193 -193.194 -194.195 -195.196 -196.197 -197.198 -198.199 -199.2 -200.201 -201.202 -202.203 -203.204 -204.205 -205.206 -206.207 -207.208 -208.209 -209.21 -210.211 -211.212 -212.213 -213.214 -214.215 -215.216 -216.217 -217.218 -218.219 -219.22 -220.221 -221.222 -222.223 -223.224 -224.225 -225.226 -226.227 -227.228 -228.229 -229.23 -230.231 -231.232 -232.233 -233.234 -234.235 -235.236 -236.237 -237.238 -238.239 -239.24 -240.241 -241.242 -242.243 -243.244 -244.245 -245.246 -246.247 -247.248 -248.249 -249.25 -250.251 -251.252 -252.253 -253.254 -254.255 -255.256 -256.257 -257.258 -258.259 -259.26 -260.261 -261.262 -262.263 -263.264 -264.265 -265.266 -266.267 -267.268 -268.269 -269.27 -270.271 -271.272 -272.273 -273.274 -274.275 -275.276 -276.277 -277.278 -278.279 -279.28 -280.281 -281.282 -282.283 -283.284 -284.285 -285.286 -286.287 -287.288 -288.289 -289.29 -290.291 -291.292 -292.293 -293.294 -294.295 -295.296 -296.297 -297.298 -298.299 -299.3 -300.301 -301.302 -302.303 -303.304 -304.305 -305.306 -306.307 -307.308 -308.309 -309.31 -310.311 -311.312 -312.313 -313.314 -314.315 -315.316 -316.317 -317.318 -318.319 -319.32 -320.321 -321.322 -322.323 -323.324 -324.325 -325.326 -326.327 -327.328 -328.329 -329.33 -330.331 -331.332 -332.333 -333.334 -334.335 -335.336 -336.337 -337.338 -338.339 -339.34 -340.341 -341.342 -342.343 -343.344 -344.345 -345.346 -346.347 -347.348 -348.349 -349.35 -350.351 -351.352 -352.353 -353.354 -354.355 -355.356 -356.357 -357.358 -358.359 -359.36 -360.361 -361.362 -362.363 -363.364 -364.365 -365.366 -366.367 -367.368 -368.369 -369.37 -370.371 -371.372 -372.373 -373.374 -374.375 -375.376 -376.377 -377.378 -378.379 -379.38 -380.381 -381.382 -382.383 -383.384 -384.385 -385.386 -386.387 -387.388 -388.389 -389.39 -390.391 -391.392 -392.393 -393.394 -394.395 -395.396 -396.397 -397.398 -398.399 -399.4 -400.401 -401.402 -402.403 -403.404 -404.405 -405.406 -406.407 -407.408 -408.409 -409.41 -410.411 -411.412 -412.413 -413.414 -414.415 -415.416 -416.417 -417.418 -418.419 -419.42 -420.421 -421.422 -422.423 -423.424 -424.425 -425.426 -426.427 -427.428 -428.429 -429.43 -430.431 -431.432 -432.433 -433.434 -434.435 -435.436 -436.437 -437.438 -438.439 -439.44 -440.441 -441.442 -442.443 -443.444 -444.445 -445.446 -446.447 -447.448 -448.449 -449.45 -450.451 -451.452 -452.453 -453.454 -454.455 -455.456 -456.457 -457.458 -458.459 -459.46 -460.461 -461.462 -462.463 -463.464 -464.465 -465.466 -466.467 -467.468 -468.469 -469.47 -470.471 -471.472 -472.473 -473.474 -474.475 -475.476 -476.477 -477.478 -478.479 -479.48 -480.481 -481.482 -482.483 -483.484 -484.485 -485.486 -486.487 -487.488 -488.489 -489.49 -490.491 -491.492 -492.493 -493.494 -494.495 -495.496 -496.497 -497.498 -498.499 -499.5 -500.501 -501.502 -502.503 -503.504 -504.505 -505.506 -506.507 -507.508 -508.509 -509.51 -510.511 -511.512 -512.513 -513.514 -514.515 -515.516 -516.517 -517.518 -518.519 -519.52 -520.521 -521.522 -522.523 -523.524 -524.525 -525.526 -526.527 -527.528 -528.529 -529.53 -530.531 -531.532 -532.533 -533.534 -534.535 -535.536 -536.537 -537.538 -538.539 -539.54 -540.541 -541.542 -542.543 -543.544 -544.545 -545.546 -546.547 -547.548 -548.549 -549.55 -550.551 -551.552 -552.553 -553.554 -554.555 -555.556 -556.557 -557.558 -558.559 -559.56 -560.561 -561.562 -562.563 -563.564 -564.565 -565.566 -566.567 -567.568 -568.569 -569.57 -570.571 -571.572 -572.573 -573.574 -574.575 -575.576 -576.577 -577.578 -578.579 -579.58 -580.581 -581.582 -582.583 -583.584 -584.585 -585.586 -586.587 -587.588 -588.589 -589.59 -590.591 -591.592 -592.593 -593.594 -594.595 -595.596 -596.597 -597.598 -598.599 -599.6 -600.601 -601.602 -602.603 -603.604 -604.605 -605.606 -606.607 -607.608 -608.609 -609.61 -610.611 -611.612 -612.613 -613.614 -614.615 -615.616 -616.617 -617.618 -618.619 -619.62 -620.621 -621.622 -622.623 -623.624 -624.625 -625.626 -626.627 -627.628 -628.629 -629.63 -630.631 -631.632 -632.633 -633.634 -634.635 -635.636 -636.637 -637.638 -638.639 -639.64 -640.641 -641.642 -642.643 -643.644 -644.645 -645.646 -646.647 -647.648 -648.649 -649.65 -650.651 -651.652 -652.653 -653.654 -654.655 -655.656 -656.657 -657.658 -658.659 -659.66 -660.661 -661.662 -662.663 -663.664 -664.665 -665.666 -666.667 -667.668 -668.669 -669.67 -670.671 -671.672 -672.673 -673.674 -674.675 -675.676 -676.677 -677.678 -678.679 -679.68 -680.681 -681.682 -682.683 -683.684 -684.685 -685.686 -686.687 -687.688 -688.689 -689.69 -690.691 -691.692 -692.693 -693.694 -694.695 -695.696 -696.697 -697.698 -698.699 -699.7 -700.701 -701.702 -702.703 -703.704 -704.705 -705.706 -706.707 -707.708 -708.709 -709.71 -710.711 -711.712 -712.713 -713.714 -714.715 -715.716 -716.717 -717.718 -718.719 -719.72 -720.721 -721.722 -722.723 -723.724 -724.725 -725.726 -726.727 -727.728 -728.729 -729.73 -730.731 -731.732 -732.733 -733.734 -734.735 -735.736 -736.737 -737.738 -738.739 -739.74 -740.741 -741.742 -742.743 -743.744 -744.745 -745.746 -746.747 -747.748 -748.749 -749.75 -750.751 -751.752 -752.753 -753.754 -754.755 -755.756 -756.757 -757.758 -758.759 -759.76 -760.761 -761.762 -762.763 -763.764 -764.765 -765.766 -766.767 -767.768 -768.769 -769.77 -770.771 -771.772 -772.773 -773.774 -774.775 -775.776 -776.777 -777.778 -778.779 -779.78 -780.781 -781.782 -782.783 -783.784 -784.785 -785.786 -786.787 -787.788 -788.789 -789.79 -790.791 -791.792 -792.793 -793.794 -794.795 -795.796 -796.797 -797.798 -798.799 -799.8 -800.801 -801.802 -802.803 -803.804 -804.805 -805.806 -806.807 -807.808 -808.809 -809.81 -810.811 -811.812 -812.813 -813.814 -814.815 -815.816 -816.817 -817.818 -818.819 -819.82 -820.821 -821.822 -822.823 -823.824 -824.825 -825.826 -826.827 -827.828 -828.829 -829.83 -830.831 -831.832 -832.833 -833.834 -834.835 -835.836 -836.837 -837.838 -838.839 -839.84 -840.841 -841.842 -842.843 -843.844 -844.845 -845.846 -846.847 -847.848 -848.849 -849.85 -850.851 -851.852 -852.853 -853.854 -854.855 -855.856 -856.857 -857.858 -858.859 -859.86 -860.861 -861.862 -862.863 -863.864 -864.865 -865.866 -866.867 -867.868 -868.869 -869.87 -870.871 -871.872 -872.873 -873.874 -874.875 -875.876 -876.877 -877.878 -878.879 -879.88 -880.881 -881.882 -882.883 -883.884 -884.885 -885.886 -886.887 -887.888 -888.889 -889.89 -890.891 -891.892 -892.893 -893.894 -894.895 -895.896 -896.897 -897.898 -898.899 -899.9 -900.901 -901.902 -902.903 -903.904 -904.905 -905.906 -906.907 -907.908 -908.909 -909.91 -910.911 -911.912 -912.913 -913.914 -914.915 -915.916 -916.917 -917.918 -918.919 -919.92 -920.921 -921.922 -922.923 -923.924 -924.925 -925.926 -926.927 -927.928 -928.929 -929.93 -930.931 -931.932 -932.933 -933.934 -934.935 -935.936 -936.937 -937.938 -938.939 -939.94 -940.941 -941.942 -942.943 -943.944 -944.945 -945.946 -946.947 -947.948 -948.949 -949.95 -950.951 -951.952 -952.953 -953.954 -954.955 -955.956 -956.957 -957.958 -958.959 -959.96 -960.961 -961.962 -962.963 -963.964 -964.965 -965.966 -966.967 -967.968 -968.969 -969.97 -970.971 -971.972 -972.973 -973.974 -974.975 -975.976 -976.977 -977.978 -978.979 -979.98 -980.981 -981.982 -982.983 -983.984 -984.985 -985.986 -986.987 -987.988 -988.989 -989.99 -990.991 -991.992 -992.993 -993.994 -994.995 -995.996 -996.997 -997.998 -998.999 -999.1 -1000.1001 -1001.1002 -1002.1003 -1003.1004 -1004.1005 -1005.1006 -1006.1007 -1007.1008 -1008.1009 -1009.101 -1010.1011 -1011.1012 -1012.1013 -1013.1014 -1014.1015 -1015.1016 -1016.1017 -1017.1018 -1018.1019 -1019.102 -1020.1021 -1021.1022 -1022.1023 -1023.1024 -1024.1025 -1025.1026 -1026.1027 -1027.1028 -1028.1029 -1029.103 -1030.1031 -1031.1032 -1032.1033 -1033.1034 -1034.1035 -1035.1036 -1036.1037 -1037.1038 -1038.1039 -1039.104 -1040.1041 -1041.1042 -1042.1043 -1043.1044 -1044.1045 -1045.1046 -1046.1047 -1047.1048 -1048.1049 -1049.105 -1050.1051 -1051.1052 -1052.1053 -1053.1054 -1054.1055 -1055.1056 -1056.1057 -1057.1058 -1058.1059 -1059.106 -1060.1061 -1061.1062 -1062.1063 -1063.1064 -1064.1065 -1065.1066 -1066.1067 -1067.1068 -1068.1069 -1069.107 -1070.1071 -1071.1072 -1072.1073 -1073.1074 -1074.1075 -1075.1076 -1076.1077 -1077.1078 -1078.1079 -1079.108 -1080.1081 -1081.1082 -1082.1083 -1083.1084 -1084.1085 -1085.1086 -1086.1087 -1087.1088 -1088.1089 -1089.109 -1090.1091 -1091.1092 -1092.1093 -1093.1094 -1094.1095 -1095.1096 -1096.1097 -1097.1098 -1098.1099 -1099.11 -1100.1101 -1101.1102 -1102.1103 -1103.1104 -1104.1105 -1105.1106 -1106.1107 -1107.1108 -1108.1109 -1109.111 -1110.1111 -1111.1112 -1112.1113 -1113.1114 -1114.1115 -1115.1116 -1116.1117 -1117.1118 -1118.1119 -1119.112 -1120.1121 -1121.1122 -1122.1123 -1123.1124 -1124.1125 -1125.1126 -1126.1127 -1127.1128 -1128.1129 -1129.113 -1130.1131 -1131.1132 -1132.1133 -1133.1134 -1134.1135 -1135.1136 -1136.1137 -1137.1138 -1138.1139 -1139.114 -1140.1141 -1141.1142 -1142.1143 -1143.1144 -1144.1145 -1145.1146 -1146.1147 -1147.1148 -1148.1149 -1149.115 -1150.1151 -1151.1152 -1152.1153 -1153.1154 -1154.1155 -1155.1156 -1156.1157 -1157.1158 -1158.1159 -1159.116 -1160.1161 -1161.1162 -1162.1163 -1163.1164 -1164.1165 -1165.1166 -1166.1167 -1167.1168 -1168.1169 -1169.117 -1170.1171 -1171.1172 -1172.1173 -1173.1174 -1174.1175 -1175.1176 -1176.1177 -1177.1178 -1178.1179 -1179.118 -1180.1181 -1181.1182 -1182.1183 -1183.1184 -1184.1185 -1185.1186 -1186.1187 -1187.1188 -1188.1189 -1189.119 -1190.1191 -1191.1192 -1192.1193 -1193.1194 -1194.1195 -1195.1196 -1196.1197 -1197.1198 -1198.1199 -1199.12 -1200.1201 -1201.1202 -1202.1203 -1203.1204 -1204.1205 -1205.1206 -1206.1207 -1207.1208 -1208.1209 -1209.121 -1210.1211 -1211.1212 -1212.1213 -1213.1214 -1214.1215 -1215.1216 -1216.1217 -1217.1218 -1218.1219 -1219.122 -1220.1221 -1221.1222 -1222.1223 -1223.1224 -1224.1225 -1225.1226 -1226.1227 -1227.1228 -1228.1229 -1229.123 -1230.1231 -1231.1232 -1232.1233 -1233.1234 -1234.1235 -1235.1236 -1236.1237 -1237.1238 -1238.1239 -1239.124 -1240.1241 -1241.1242 -1242.1243 -1243.1244 -1244.1245 -1245.1246 -1246.1247 -1247.1248 -1248.1249 -1249.125 -1250.1251 -1251.1252 -1252.1253 -1253.1254 -1254.1255 -1255.1256 -1256.1257 -1257.1258 -1258.1259 -1259.126 -1260.1261 -1261.1262 -1262.1263 -1263.1264 -1264.1265 -1265.1266 -1266.1267 -1267.1268 -1268.1269 -1269.127 -1270.1271 -1271.1272 -1272.1273 -1273.1274 -1274.1275 -1275.1276 -1276.1277 -1277.1278 -1278.1279 -1279.128 -1280.1281 -1281.1282 -1282.1283 -1283.1284 -1284.1285 -1285.1286 -1286.1287 -1287.1288 -1288.1289 -1289.129 -1290.1291 -1291.1292 -1292.1293 -1293.1294 -1294.1295 -1295.1296 -1296.1297 -1297.1298 -1298.1299 -1299.13 -1300.1301 -1301.1302 -1302.1303 -1303.1304 -1304.1305 -1305.1306 -1306.1307 -1307.1308 -1308.1309 -1309.131 -1310.1311 -1311.1312 -1312.1313 -1313.1314 -1314.1315 -1315.1316 -1316.1317 -1317.1318 -1318.1319 -1319.132 -1320.1321 -1321.1322 -1322.1323 -1323.1324 -1324.1325 -1325.1326 -1326.1327 -1327.1328 -1328.1329 -1329.133 -1330.1331 -1331.1332 -1332.1333 -1333.1334 -1334.1335 -1335.1336 -1336.1337 -1337.1338 -1338.1339 -1339.134 -1340.1341 -1341.1342 -1342.1343 -1343.1344 -1344.1345 -1345.1346 -1346.1347 -1347.1348 -1348.1349 -1349.135 -1350.1351 -1351.1352 -1352.1353 -1353.1354 -1354.1355 -1355.1356 -1356.1357 -1357.1358 -1358.1359 -1359.136 -1360.1361 -1361.1362 -1362.1363 -1363.1364 -1364.1365 -1365.1366 -1366.1367 -1367.1368 -1368.1369 -1369.137 -1370.1371 -1371.1372 -1372.1373 -1373.1374 -1374.1375 -1375.1376 -1376.1377 -1377.1378 -1378.1379 -1379.138 -1380.1381 -1381.1382 -1382.1383 -1383.1384 -1384.1385 -1385.1386 -1386.1387 -1387.1388 -1388.1389 -1389.139 -1390.1391 -1391.1392 -1392.1393 -1393.1394 -1394.1395 -1395.1396 -1396.1397 -1397.1398 -1398.1399 -1399.14 -1400.1401 -1401.1402 -1402.1403 -1403.1404 -1404.1405 -1405.1406 -1406.1407 -1407.1408 -1408.1409 -1409.141 -1410.1411 -1411.1412 -1412.1413 -1413.1414 -1414.1415 -1415.1416 -1416.1417 -1417.1418 -1418.1419 -1419.142 -1420.1421 -1421.1422 -1422.1423 -1423.1424 -1424.1425 -1425.1426 -1426.1427 -1427.1428 -1428.1429 -1429.143 -1430.1431 -1431.1432 -1432.1433 -1433.1434 -1434.1435 -1435.1436 -1436.1437 -1437.1438 -1438.1439 -1439.144 -1440.1441 -1441.1442 -1442.1443 -1443.1444 -1444.1445 -1445.1446 -1446.1447 -1447.1448 -1448.1449 -1449.145 -1450.1451 -1451.1452 -1452.1453 -1453.1454 -1454.1455 -1455.1456 -1456.1457 -1457.1458 -1458.1459 -1459.146 -1460.1461 -1461.1462 -1462.1463 -1463.1464 -1464.1465 -1465.1466 -1466.1467 -1467.1468 -1468.1469 -1469.147 -1470.1471 -1471.1472 -1472.1473 -1473.1474 -1474.1475 -1475.1476 -1476.1477 -1477.1478 -1478.1479 -1479.148 -1480.1481 -1481.1482 -1482.1483 -1483.1484 -1484.1485 -1485.1486 -1486.1487 -1487.1488 -1488.1489 -1489.149 -1490.1491 -1491.1492 -1492.1493 -1493.1494 -1494.1495 -1495.1496 -1496.1497 -1497.1498 -1498.1499 -1499.15 -1500.1501 -1501.1502 -1502.1503 -1503.1504 -1504.1505 -1505.1506 -1506.1507 -1507.1508 -1508.1509 -1509.151 -1510.1511 -1511.1512 -1512.1513 -1513.1514 -1514.1515 -1515.1516 -1516.1517 -1517.1518 -1518.1519 -1519.152 -1520.1521 -1521.1522 -1522.1523 -1523.1524 -1524.1525 -1525.1526 -1526.1527 -1527.1528 -1528.1529 -1529.153 -1530.1531 -1531.1532 -1532.1533 -1533.1534 -1534.1535 -1535.1536 -1536.1537 -1537.1538 -1538.1539 -1539.154 -1540.1541 -1541.1542 -1542.1543 -1543.1544 -1544.1545 -1545.1546 -1546.1547 -1547.1548 -1548.1549 -1549.155 -1550.1551 -1551.1552 -1552.1553 -1553.1554 -1554.1555 -1555.1556 -1556.1557 -1557.1558 -1558.1559 -1559.156 -1560.1561 -1561.1562 -1562.1563 -1563.1564 -1564.1565 -1565.1566 -1566.1567 -1567.1568 -1568.1569 -1569.157 -1570.1571 -1571.1572 -1572.1573 -1573.1574 -1574.1575 -1575.1576 -1576.1577 -1577.1578 -1578.1579 -1579.158 -1580.1581 -1581.1582 -1582.1583 -1583.1584 -1584.1585 -1585.1586 -1586.1587 -1587.1588 -1588.1589 -1589.159 -1590.1591 -1591.1592 -1592.1593 -1593.1594 -1594.1595 -1595.1596 -1596.1597 -1597.1598 -1598.1599 -1599.16 -1600.1601 -1601.1602 -1602.1603 -1603.1604 -1604.1605 -1605.1606 -1606.1607 -1607.1608 -1608.1609 -1609.161 -1610.1611 -1611.1612 -1612.1613 -1613.1614 -1614.1615 -1615.1616 -1616.1617 -1617.1618 -1618.1619 -1619.162 -1620.1621 -1621.1622 -1622.1623 -1623.1624 -1624.1625 -1625.1626 -1626.1627 -1627.1628 -1628.1629 -1629.163 -1630.1631 -1631.1632 -1632.1633 -1633.1634 -1634.1635 -1635.1636 -1636.1637 -1637.1638 -1638.1639 -1639.164 -1640.1641 -1641.1642 -1642.1643 -1643.1644 -1644.1645 -1645.1646 -1646.1647 -1647.1648 -1648.1649 -1649.165 -1650.1651 -1651.1652 -1652.1653 -1653.1654 -1654.1655 -1655.1656 -1656.1657 -1657.1658 -1658.1659 -1659.166 -1660.1661 -1661.1662 -1662.1663 -1663.1664 -1664.1665 -1665.1666 -1666.1667 -1667.1668 -1668.1669 -1669.167 -1670.1671 -1671.1672 -1672.1673 -1673.1674 -1674.1675 -1675.1676 -1676.1677 -1677.1678 -1678.1679 -1679.168 -1680.1681 -1681.1682 -1682.1683 -1683.1684 -1684.1685 -1685.1686 -1686.1687 -1687.1688 -1688.1689 -1689.169 -1690.1691 -1691.1692 -1692.1693 -1693.1694 -1694.1695 -1695.1696 -1696.1697 -1697.1698 -1698.1699 -1699.17 -1700.1701 -1701.1702 -1702.1703 -1703.1704 -1704.1705 -1705.1706 -1706.1707 -1707.1708 -1708.1709 -1709.171 -1710.1711 -1711.1712 -1712.1713 -1713.1714 -1714.1715 -1715.1716 -1716.1717 -1717.1718 -1718.1719 -1719.172 -1720.1721 -1721.1722 -1722.1723 -1723.1724 -1724.1725 -1725.1726 -1726.1727 -1727.1728 -1728.1729 -1729.173 -1730.1731 -1731.1732 -1732.1733 -1733.1734 -1734.1735 -1735.1736 -1736.1737 -1737.1738 -1738.1739 -1739.174 -1740.1741 -1741.1742 -1742.1743 -1743.1744 -1744.1745 -1745.1746 -1746.1747 -1747.1748 -1748.1749 -1749.175 -1750.1751 -1751.1752 -1752.1753 -1753.1754 -1754.1755 -1755.1756 -1756.1757 -1757.1758 -1758.1759 -1759.176 -1760.1761 -1761.1762 -1762.1763 -1763.1764 -1764.1765 -1765.1766 -1766.1767 -1767.1768 -1768.1769 -1769.177 -1770.1771 -1771.1772 -1772.1773 -1773.1774 -1774.1775 -1775.1776 -1776.1777 -1777.1778 -1778.1779 -1779.178 -1780.1781 -1781.1782 -1782.1783 -1783.1784 -1784.1785 -1785.1786 -1786.1787 -1787.1788 -1788.1789 -1789.179 -1790.1791 -1791.1792 -1792.1793 -1793.1794 -1794.1795 -1795.1796 -1796.1797 -1797.1798 -1798.1799 -1799.18 -1800.1801 -1801.1802 -1802.1803 -1803.1804 -1804.1805 -1805.1806 -1806.1807 -1807.1808 -1808.1809 -1809.181 -1810.1811 -1811.1812 -1812.1813 -1813.1814 -1814.1815 -1815.1816 -1816.1817 -1817.1818 -1818.1819 -1819.182 -1820.1821 -1821.1822 -1822.1823 -1823.1824 -1824.1825 -1825.1826 -1826.1827 -1827.1828 -1828.1829 -1829.183 -1830.1831 -1831.1832 -1832.1833 -1833.1834 -1834.1835 -1835.1836 -1836.1837 -1837.1838 -1838.1839 -1839.184 -1840.1841 -1841.1842 -1842.1843 -1843.1844 -1844.1845 -1845.1846 -1846.1847 -1847.1848 -1848.1849 -1849.185 -1850.1851 -1851.1852 -1852.1853 -1853.1854 -1854.1855 -1855.1856 -1856.1857 -1857.1858 -1858.1859 -1859.186 -1860.1861 -1861.1862 -1862.1863 -1863.1864 -1864.1865 -1865.1866 -1866.1867 -1867.1868 -1868.1869 -1869.187 -1870.1871 -1871.1872 -1872.1873 -1873.1874 -1874.1875 -1875.1876 -1876.1877 -1877.1878 -1878.1879 -1879.188 -1880.1881 -1881.1882 -1882.1883 -1883.1884 -1884.1885 -1885.1886 -1886.1887 -1887.1888 -1888.1889 -1889.189 -1890.1891 -1891.1892 -1892.1893 -1893.1894 -1894.1895 -1895.1896 -1896.1897 -1897.1898 -1898.1899 -1899.19 -1900.1901 -1901.1902 -1902.1903 -1903.1904 -1904.1905 -1905.1906 -1906.1907 -1907.1908 -1908.1909 -1909.191 -1910.1911 -1911.1912 -1912.1913 -1913.1914 -1914.1915 -1915.1916 -1916.1917 -1917.1918 -1918.1919 -1919.192 -1920.1921 -1921.1922 -1922.1923 -1923.1924 -1924.1925 -1925.1926 -1926.1927 -1927.1928 -1928.1929 -1929.193 -1930.1931 -1931.1932 -1932.1933 -1933.1934 -1934.1935 -1935.1936 -1936.1937 -1937.1938 -1938.1939 -1939.194 -1940.1941 -1941.1942 -1942.1943 -1943.1944 -1944.1945 -1945.1946 -1946.1947 -1947.1948 -1948.1949 -1949.195 -1950.1951 -1951.1952 -1952.1953 -1953.1954 -1954.1955 -1955.1956 -1956.1957 -1957.1958 -1958.1959 -1959.196 -1960.1961 -1961.1962 -1962.1963 -1963.1964 -1964.1965 -1965.1966 -1966.1967 -1967.1968 -1968.1969 -1969.197 -1970.1971 -1971.1972 -1972.1973 -1973.1974 -1974.1975 -1975.1976 -1976.1977 -1977.1978 -1978.1979 -1979.198 -1980.1981 -1981.1982 -1982.1983 -1983.1984 -1984.1985 -1985.1986 -1986.1987 -1987.1988 -1988.1989 -1989.199 -1990.1991 -1991.1992 -1992.1993 -1993.1994 -1994.1995 -1995.1996 -1996.1997 -1997.1998 -1998.1999 -1999.2 diff --git a/flink-connectors/flink-orc/src/test/resources/demo-11-none.orc b/flink-connectors/flink-orc/src/test/resources/demo-11-none.orc deleted file mode 100644 index 1d1d714a846259ec5b2b0471e55eec94efd7671d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5147970 zcmeFa0dP}wzxVk+NtzP^LK;vG)JM49WqD_3M|PZdo$c<9Gj|TF`_3-xJNLdUJ3H&n z%f5Df?{bA5c6rr(_Sx2mu>>eJV1Q6-janc;sX=Ou8cL8_qSP9oK!94K)*7IcDgh$a z{hm|oQ%Y#lOWM@>of|(VoFq;1%{kvY^UVKqe*eSYtJXZb(Na3!dcV~mgnb(B+N~CY zt$eZ4v?V#&Iz!(*6}umvWR;$12X z7y3-reEsps+6&DVVH1kTY*(>lD3+Y~&t)o>%FM-*+fpofti_UdO0iTqwOI1milu6M zvE-joEY&+?@B0WPxr|@?2qn2a0*po;gggj&5V8r` zgls}KA)Am*$R=bHvI*IQY(h36n~+U-{kQ*##vr7Hi~eEBQdyW}ktYkIEQBm1Sr}wt z=1vB}gMlXko8foeiZ>=to+itaSB|K(p7p#R;||6$pRe^>PMt@Y^Y%76N&qGyW! z^{Rh-cG;>I{-NmKtz7Xm?%Z0Wu8K}s6)n?LQ9n}^RdZBPI$sstkEr7GW2$I)TopT? zP(}G-Ra{@9iuR{e(Nv_0+GkaiGI_{_hb`}hC&e=Yy-S3ak2Z~cfve5z4; zsBp+ACfJ)=%@#8R3|Zew%1nK0P#(~?##>vHGGG69o-#|{nw7cwR#v9#Tg$EOBy44i zWRsNbl77da)M~FZ*bIshTO+mx{Q=`|4;XJfU{vz-2TaPj$@(iyiba2gNm2C&@{~OW z?2v3`<$3)HX1qfClV;^XeZQ=%*0+{h+eyl5yTKy#|4ewD!LBh+r`*5S)c+hj_cw*+ zHnGLMSQlHGu7CZvgLIjDmS{>ZrI%=0&R(1~tVCniC7LOkL?c6@F(~Z|Gqq^ki=nOj zYiKin4{he(K%4m=q0RhH&}RN;Xfyu{w3+`E+RVR&HuJwhn|T`A%>NE;=HEk``9Gk| z{1dd970_n>PiQl%pw0Z>&}RMvw3+9j&HO*mX8vDjGyfm7nGy-!j05Oh12|p*oNEEs zI-qO=;NA#$UIo0HfXdeZ-xi>Hn^Y+Iw@X~Fzyz286JP>NfC(@GCcp%k025#WOn?b6 z0Vco%m_W`Gpj7kQ0);+{h$}X@c8zP-xOR>Oj{!tKXYR62hQFLd;t~o3B zS$q|etu`SkO=ph%SJ^ge?lzvLBpyp)Wf;)kb9|-RO8f$<^EzleQqVEB%bwI2hXbS@IkARL4kZ1t9!a&a_ zK(Y}?H37X5z;O(4egU|?1j>#B?vsG$6yQArRGtNV=Yi_40RPuO%{M^cJD~1+I5)u` z=-h~$uczP#U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WbZ+R}{5Hg=Z-x(76>(Kj z&Q}%jAml;FgOH1SxyYAI$R=bHvI*IQY(h36n~+V&CQRepOth8eoIk?(BfsnUBmD3j zrmeIt7{>V;YTHf%a$rpxVuB zU8BidySe`^+RfoyXgAuN8_AZ1c{aCcH-Us?Yp6FUO~DWTA=}zb=Y{kGZL0RH8G4{i zdHEpi#)P>(Rm~Y|H~3OeSS|v=E+EtmgnNL-pMXdbXub|aQ$XuYAl3`Cl^CGdIDn2d zK*9-hl>$93Ah{k$l>xowfMXNjd<}4I0m`-k?(Kl*b-?=uQ28d{dkd(32k`FzYIXsE z-9X)5Ah-_*y$gi*1C0j^g;L~w16MRK0Vco%m;e)C0!)AjFaajO1egF5U;<2l2`~XB zkn;qn-B7!sb|XZea@2P$aCIA3w{dkFSGRGUFPo4}$R=bHvI*IQY(h36n~+V&Cd_#g zrqOOV>ID6p3q3m{L#ESi@TH)z1cAs$KywI)HUO<*AodB+)(FI# zfQ|@|_#EhJ26|e6WE4o90D4;i#~HwR7I2*h%Dw{JUjv?R0PlA|<@bQ^2cY@_;J*aa zTm}MHfVv-n;8h@W4G7-=8h-{NzW~j@0?`+Z_6S4`}glxhz+KqwZVK^R!6IM83B`XtFXfMHh7$?V47PXsvs3fX5{g^5m z9#_TACsa|sSQXcosG|KTRWuc;qW0N|s?n$;{qC}+CH0Fp1MgrS%@76-9=b;f2Lo#| zux|c2E={3k&RFi@M(bwWSvLWLZq=9-ZK%ztS~s(-1|jUzEv57AQ!P=a(bj$f*3HfG zm!xcKHw|~0cGG=xyJCGv{}>8B#>3OCrYBdfeA+HAS+RV@%I8)n4ci83Hx`>!X+M>* zc7tyPg{2gTxq!CyK)ej-C$?p8~B%f!Jq2+c6;i z1<>&&w420nYBz#wI+y?xU;<2l2`~XBzyz286JP>NfC(@GCcp%kK+Y4Ob~8{n5Q1wu zxVnw2+qk-otJ~OwY(h36n~+V&CS((`3E6~fLN+0rkWHBLCQPH*W$ z7HCiYgK9SgFj~wu!>u7V<7?Q>zTvbRO~;Wi^rdg;%_L>D-Cz;=e7IZ z`k#a6TJ!8iNG{gfw43+|qpjO#Qku@+^k!YVslS+KovGV3`I_W2#nLI;Fp*0sS$A^w z$g~@)ZHm%RVa#;e4Zal=mR2Bf8t94vJ?DUA8<1)TdgFlOJHYup;Q9e5y8yT^0iMf% z_X<$?BjCFVR9^%9H-MU-fxs_7-LF9KMH3X8mw@ogK;tVwWG&FV4v205S~mi*SAn)o zK>RhJV+)Yj26Sx)dR_;TZNfC(@GCcp%k z025#WOn}-AwHr>m;j|l0yD^yto(EiF$t9LtV#y_zIbUMQgOCRy4?_OGWB$KmHX)mk zO~@u>6Q;F5F7yArFY~9p#KBQKi^tDdoJH-XfD>*;G~EZ@&7fQUm}oc|%(@wY zFO!K`vs80xrVMdgraemOX57uXsU1$b>62};E#36^Otz9MkZwYqfvih6-HG(lO~bMM zGl!IJN)8U1bt53%w68T;bU9|Qh_K+Pc_a2TjN0t7z=LPvq{XF%gIAo2y!{3Q@Q z4z!*GVyA$%GeG<-&~Y9}dm3xQ$K7CS((` z3E6~fLN+0rkWI)YWD~LpbKZn$tQ!*-%8#Fg^0MI1VdT%jc{e-}WO1)2LKdwXRdmX# zXql#p`kAVznxl%+`KsuCL=~qWQ$@q$Dyehw@7O_jrv;;`f7zj=)-MdltWZknc!o4RGH8JU;{8Ux3PA0pE*xP;6cT{4WDFuKSZvlySfUX@t&n_Uj8%XU1diMd20N{KNaMb~2^?*AF zcs|N2l)RxlUPqJ(FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^xoF_o-hT09a8xFhS zup17$VH2_m*@SFDHX)mkO~@u>6S4`}gls}K;W%r;G}?{ej1tZ$p?XEvhpx{AbA7U? z-OPhdqKc*>Rn$JKimj_vQSuk4IBO)hIO~D>%|KNJKs~1rYSKU*V^KOQWS}RKEfYu0 zNWB?56=k*l#nn5En1SDYls0 z;#rq&?l$bEBpyZ5@FGoa=e z5cmS9`w|Eq2SO)-@F}423=laBG@l2eUjeOO1F>&_w(o%W_dv%FK;i<>bqVOX3?#1r zsULyftAOJ=;7kFon?PAF;4U%a^Em+T8lch%_)39lm$^{#uQ%s>IRQTa6JP>NfC(@G zCcp%k025#WOn?b60Vco%m;e)C0+epJtcc5sxU6WLEi2+BrFlu|oL^F!2O$qa9)z6m zzzGj*LN+0rkWI)YWD~LpbKZn$q#N@DTWU^&h6W7{8X7dLwJb_EGH2XmJl`iDQqmB{ zN5QB$I7@{wgLyZDhr*1K_R{}nb^j$NZ%XOLHpHZx97#964Q5;ISx7fu?@nc1y6L=- ze$q|Vo;5>Hy6N_rtkbOqA?){AO6S`xm?mUWN<30p6DMDP;$-cKW{a>1MZjR$pcyP1 z0spH&%_bo58c??d2yO#H+kx=wK;s)g0%1M$5;$37tOF3`0f z=s5r+-v?430KFdqju7B%09;|9>=VG<2zZ(RZv?3P9Pl**)h&QO3e=nc0NfC(@GCcp%k025#WOd#h8(7K^@L+ggt4Xqnq)|^eq zCS((`3E6~fLN+0rkWI)YWD~Lp*@WY)3Da0NMk)$a6sRc3f+vE+6Ja8q2wAjl7I3^x z#-n`}i0(&|7XJE z2D`>QopS$PQ~z`DTx*`)2tma^8*04)fn!;P$2I~y{p{aQ2;Yn8M z3A?mNDPKHj??St@RNFgGf9@RZxwEa4AD#Qcs<{h`o>^5?WS5t$SiWNAb1Rg)xi<~T zqeHtlEfs^@8xw+Pk7e%O;A_ESIS&NC0zzK{;ctM(?|{hnK=Thk^a9X&35Z<=+O7cc zAAydmK;jzEbpz=68A$#Dq<#f@UzFk5tO1-(z*P#Axd8Wiz*7cz%YjNa;M)vTdjS7d zpvDUXDuB95AXo*2d_eeZps`vmlp;IjoUbb22VeqBfC(@GCcp%k025#WOn?b60Vco% zm;e)C0!)DJ4ObU&brDw=jkDE7ys$JcES>WUOYYi7~C$%jv(iqns&qTz8>?0iBM<%?BueTgdC zpHf9rkt%ARRq5yS`8l#^Suge5Hv{irCe5gtI0Zt=K)p$$+bq@881pFImT7;P-p#o4 zZUP3~r!nh(O*-$U&$r38cN5+v+gh%{yLsy+f7ZR5hP%wX>AtyLu|A}GJBsG*Ot+ez zT)Fb;zY06$<%7H%v(2J3ykW?A<_*3U9G2ZcWG~RX4~V`CwC)FD2Y|Nsf%peN$A>`T zW1#CG&~pe#9tKiJfZk65M+9(w4!D|uvKGJ{1w1DJZ!1uF8t}z{>T`g<4X9}c0&$@3 zTOim0ggSw60%*JlM7n_HZXntNwEhI|CYGdk!tO?V2 zH+h`)Aq!6XFjKqWiNF(KB1wo@^ll0@l|&Wwn}Mndpi@>w%QWnni5+uPQ956xnxkt; z^PnXa3q+J&z3JC*mSx+)S(K4mA&b*uGHQCmF z#G*8rAn9$~F|1QnK#k?2&ml7MoS6`Zja#244#f%MBp* zGtl-65dRhEc+mpK<|Ux(WuWI3Ah{Mutpj>D0FH9N=>}Yzfie%^-U@iUfVTputOR^j zK(!C>zYWw>1A(1DogWD90YWuExE5#(0Fn2A<~kr+53~k>*hfHH2#7ZT9brqMl=#HL zONlZ8Ccp%k025#WOn?b60Vco%m;e)C0!)AjFaaiz^91PKaAgr!7I9_KI9pl7%S!XI z(mB7ZG!H@^gggj2<$+Tk*o16CHX)mkO~@u>6Xv`L(|9*jPN@E6gKqg_;^Aa4?Pg>~&1%h=vEIu~)m~2PX53jfwZmCAqciO$e#Bzy z-ZV*R`Yv`R>()(oBE5CfaBTm~A+4K+ZG)!WSZ!03l3m7(r`_Od!C?6mNE`*aJ_CA= z0m(0b)R#c-alp|EI8Otv7*KW&aJK=TcEB44D!&DM9YA#_;7$%4`ar3(Y43!VtF;E6DCPJ}F4Hym%1*=U~v(f-sR z-Dc1~myM2nlP?Sz7&rH>&rB&6Be<>Bo=oFr+!;4}a%0@|Y?@>%iNm;QY`l_n(ZIX`JB5Xntuva!|_R4EOY75Z24RBNd z&Pu>l1(f*!_uGJ{8u0D}D*b?O4^UkL_-lch01$W&sH+2l^*|^Hgg*irLqMbfXbuC> zPk`1&Al3x5MS%F{Ku0r>XaTySK+g#v*$Sji1HCc8aUO7f1>?r`HH{m=NfC(@GCcp%k025#WOd#h8(7556BCaXonxb*Griho5=H;YwemQ9#gggj& z5OTT$r#r9-*@SFDHX)mkO~@w9c@w5FZn(yN;;gZ!F(C{7nsn%lJP~jrWYM^p2a`k< zO+~7xeO47)SF57rFR*adNUCUA57ci4sw#j^8M~%o&rIx?ql(h`s_1@16{jClMZ@Fj zA7EZiA-?-S&t#G2oD3d&GjLC~Bpfv(>t+PP%&>M%zGl>jVQ=%a=VVj7fnQWCOvB#h zY0nw{<7M|bg+f@(aBPGk>sv`F(6DqY>{k|a#+&u7?fJ=l?IzZF=A`P)}TLN{Otkbtp|)sp8kMIIX78*^Hl}>08D@hFaajO1egF5U;<2l2`~XBzyz286JP>NfC6S4`}gls}KA)Am*$R=bHvI*IQr1p&cp}Jx zC&I)z5whss$bVp$W~3fbKHQ`sRFi^HQ*o9Gk-@~9(Q;9i-9@ljnwcjvtVlDWClvlZ zt(!jE#&XNP$+2~_`j&Omx6PJy{;rfXX_+R}WMN0slupO$Z1y0Ciy?_z4hd1j0=~V+4qN4m39d z(H5XJ3dBwTZLL83G|&+P66b)fHlU{+NXCKGw?JUI1K|fU?Vg`wHOs5%69G zDz5>)8$k8Xux|Xn(7NHo8z#U6m;e)C0!)AjFaajO1egF5U;<2l2`~XBkoyE^-EdtI z*A;PH(KuUI#0yLF!qPdvurv=s9)vsyIp=|M9@vCzLN+0rkWI)YWE1AR3Da0NTxUOV z*4fjVkOhBDdUSa_5pW`8(Yjf{DK{BU_*sAnKm9{gFnVbS0|yT+%c_Kfx644h$>#8x zA->&$0lp24#t~^VS$Q(uo6INP{8=A~W5$deZHSIpc~Ez5rM(~vwmw0qNXz?9yz$#~=qz86H6lF1Nl96-$)Am9Y*N`asY2(1UgWk6#& z5OD*|n}MhYXx$3Lyg*w85U&I}s(^$K=z1IIsRoiefs`NU-2*uG0nT>;*M6Yv0N{Qf z@O%JxKLjd227CvB>O+A4Fi>*@2z(0E9i3b#1wWh2tBEoJCcp%k025#WOn?b60Vco% zm;e)C0!)AjFaaiz^8_g0P`;sjL-~gC4KHubCS((`3E6~fLN+0rkWI)YWD~Lp*@SGu zan^)sthxQbp~vDg~WBK}QxX>!p7EX5bwRr5Qq2xpx((Ko~P{Z^l}K(H)qayy<9-_Zj$` z4!mC|1e_JKO(?Rym6QT~Yfv80x5isrlQLibcb+m!-}G6+-UJMIQwgHY z^Vk}&9W36A&d{5{=aX#>-YH5`Y_T3@gn3byUrSvvVzW$dc<1g9G7J=VeKw|k^lUSO8U<(k60^t)t zV=E9j4K&Aq=sBRZ4T!Y^ZE+y}Ezr>cBszhv1kiI4NOl3KZlJdZa9jnP*8tZIpzLSB z{R`mv74W_|1(MB6fbV6X`W3*x7N}VV1U3M58-d`fKxh*Xehp~cGNn+8Y@3qvB?bHd zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l2~fP@(jqP`;?kmVwzP;Bm*&N#bAEAY z9)vsyc@T2e17|(33E6~fLN+0rkWI)Y%y|>05pTHEe&Q^(r#OMJ4>X8q5b;F7iI7F{ zrciT9R8hYfsHy-uWmU9H!=9PgF-H}p^HsVzx|cK$UQ)3zs=_!>$++|WvTRE@i!!rr zRBir@dCYFBwfCZR0~=?2SvUJ~W8H+kQ*14rux@sL-I;akrv74{b*8SwrT!dnRR2fS(K7JhRm%Sd@mR*6+omCXs!aHKA`n&AXW{u?F8a}pkoh^r~$fa zft~=6d=E&~0loEr<3qstG2l7~lpO-xhXKzK!22msc@*${22>vd{9gbyUjl*SK;20o zcnS!e0m5g2#`8erE1>ynAo>l^`W>vB*!Q$<1ebI$0Vco%m;e)C0!)AjFaajO1egF5 zU;<2l2{3`2CqV0l)(x#2S~s+AcxiJsA)Am*$R=bHvI*IQY(h36n~+V&CS((ivnEVq z-AJC;!?i69G}2*wF>B4p9JDd32k5smkOb~EUfKPDDV2E%R!3pe?i7GoLJ z+dS>>)4UmX=1uKz=8ZN7M}mOTH}__evf6I22>m}3_8aUP^K{Dndrke%!E>#7b|VBB z>nz=cnXb7oQ!UZXDYo{y`;?~j$4X4r8Tvz0@zBGQtkM&9X^~RCc+lR3c4?`$cb@*- zIofk)TPHs{_k~q+7ZyFUs;Fqd*qTIo^QPh0{+UCXH|6Dn=G|CqR;9gr1m+FC7c7

O}i`k9SFV-gx&zcZvu^P0g-or<{dzE7tp#Ji0uX1_5tyCfsXz6 z6-tQ%_vL&|0Y3l}U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WG;g@Jh--_uwrHHK zE#hURd0FY4Usjq2ArC?xgq-rgDGzKyHX)mkO~@u>6S4_&-h^q)8?LpVIBV@`PRN44 zCS5u+PXwF@Su}4r;3l)-J_VxvsX@xkpnonK9Rnv{NEui+LkTv^G(ETCV`HAAz>3 zK>QleaRW&F40Qbh^!y4WUz}PfrCyrK)eKC42`~XBzyz286JP>NfC(@GCcp%k025#W zOn?dGJOL^|-(>BN(7YLU=1q=<-Sn)PYAZPo^QQUj+N_&5-FF#wQxXp;Lz*|Wo)skz_9^vZUkJf0%e;3 z_iKP>3*g-bRBi`+uLIR@0RA_Dnzw+!J3!qIAh-(%?FPbofyRA63T&E*2xc6>2XT@khPKm*AUXR*V@M zhOBQTr9j^rln3;!@z&O)%-8>&r_9o~W@WCvm6iMTt>xBsk}};YVj5Li*Jv*_Dd#5ZuP`YV{S_uf)gQ=HhShFX;{}o}+XHVL z$EVuL-<+m2?JIA}x^~lfA^pIcsy%Cl9(dER&1ALfGR+iCrja4j7?kn{Z?|sDmc3@ zfif@Pt^hoh(+VYT)if?>U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WOd#h8(7K^@ zL+ggt4Xqnq+ni0vCS((`3E6~fLN+0rkWI)YWD~Lp*@WY)3Da0NMpt>D_s*dLJn3V^0;YLwr{Xp;l5PBa7e*iRo2t+;xnhyfeLqO|cAa(?3`xJ;D1v)+h632k9FMysef#h)@ zbrR@31vp}W^Bmx61IpR~cO3A13wS$#%1*$S0IDxSyzzHYyy3VTCcp%k025#WOn?b6 z0Vco%m;e)C0!)AjFaaiz`vfT7aA^^j7IA6OI9po8t4i~#(mB7XG!H@^gggj26Xv`L(}*`*YCmz7+Ebj61%FMdbP`VloCsMIZ}K6NsN(cvs%Usz z6+53$MfqY?TwkJ!_NP?QRHTa9XI08MeR7U0TGmTxRTK=Q87SS{dH)_YoB|;-@NP1a zWzv5BA$AsMU+u0_Z3?u9sNO)S8Gh!?SgJPxgRa<^mFMrG-e~)?Kl8?Sd77>EW1G@+ z<@;+{S8p2bGV`YU=61#UkUrB!!AzTnr&~==u3Y)FU0$+c`HGd#tx!r1nr_!@OmL7| z-jQzKIn>Y_d@pz`R{;NyK+RPka1E%t0R(>rLcajvUxCIKZSZVf0-9e2qOSn0Yk}B0 zplt&X-w1TP3M4iGU9SN>TY%&?AhjLneI0OA0Zt#_dK)OK2HZOVk00>v0V-<%UoB7_ z0Q~O(HFZFs9;gf23Z>vjwwzzilOKQyFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@G zx;J!h=-$x1p?kv%oU;kpgls}KA)Am*$R=bHvI*IQY(h36n{b>pVH)=)PZnIP&&B#? zS}QyeWWf_*;+zOsbZ-hZokSJ&n}Mndpi@>w%QWnni5+uPQ956xo};Tt^Pnab3!x4- z5@uPpJsb?7$w0ihs~$6iIWwY%mfx<~EE;p=29nS3bl^?jpqsH7c(ZRf>t=KY-UL3j z*&5d6D^1bk8}qC)bQ5N(X2MLeN>A9OMN0YNL3P7gDV9#zhPF~l)}5R^GVA6y!DhPt*Y4B)ng!yNS*cA( z)_i^EWNoL}B5XntFjzj;43>jH@DLC>41|vWjh_OMqd@a#K=c^U`UMdC5@31*E%IAUxCcp%k025#WOn?b60Vco%m;e)C0!)AjFaajO z1ah7LtsAZ@;<_TPD;j6(ig;mZURXNk7nbHh$b*mvA?G}B&I6l}O~@u>6S4`}glxi` zH(?s%C`LG*&FDNVKcJy6A+33;m?7_W+2i6G)IBx381wV+D+^=cn(6||hc!1`uK-3GgRsgX|psfms z`+$zO?S)dJ+RoJsOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l3FJHhDmPqI#6?A1 zR5Z>O74d@7yr6W>FDT7}kOv_TLe6*KdZ^AUn4Hwx@oJICj zCS<{1lMN%Bl?dEQiZc5@IWk~I&)-!0*jny_qDG3`hpLB!o z1%+i7kk}1$?FD-F0m*lP)PA7%0N|(xoI$|#5l|Ka+zo&y40t~QDjNY`6Hpxi{GS6g z%|M_9sEY!@6F{gH2%iQTV?g8_(A)+@+kw_N5c?Kr>j2`NKt}>dTm-thfSzt3*#qq+ z^%J!lPP$MfZ7eU8)`SyZm8Yxs^)A$ zHX)mkO~@u>6S4`}gls}KA)Am*$R-?TO_)ZzG0TGM^tn!7qOigfK^8m_CeDeFMeRoZ z18X!RRfzJTCG|Tucit%&H4A5{kTRHdb63x%K=Wg!jK-}%`?EA}et+i8^H#G(vRRbX z`gW4C+HSB2{XY}-8|)hMbjtmEP5sZobFF!Hqos7db(Ym2gnc^h+NWB2uG(!SpG;Sp zS{;|OZr*fWNI&qVYR{UX2i}yI4>E5A%$xR|(qQw(qSRIxtoiyEnyh^xvqjj1B4Dvx z)hw24KTVRZDkO~659|4Yofb-Dw zLdkV_dd@H9$q&E;m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28%^R*Q;@TpvEgENQ zi+EjWUROHj*Olf$$b*mvA;&y$%mbT{O~@u>6S4`}glxi`H(?s{hHLF7&RTn#6SCm1 zNtZ56Wasl|{!I&ofTUuEi6QG-Nh#2`2IT>LYrM5JDf9Jz z=P9%Fty!6?Z)N3veQUY3ouo{+%FtUR<*=mRF(|d#D-AY-V#L;ntwDdl_}c@HRLJ**ZgaSf;{Zd3cgldcrO(Qpy((+Ply$E!FnU)1Ny>d+u!OTSZstux^61d!P1Poo)vo{jDVk3s!>2JQE&t+n@x}rTsi8i-c!RG6 ziKP*6H34N2!2LPkX$HJ4KxGv0odBv^0sm>BCI$q~0d;LaupJ1+f$+CLV+RoF1ey~- z^divO1;o06wjLn<6VQm^b0tBcQBM@ zpmcNR{d?4K3WOm8@8+LlqrfOB7KVGcWkIDGk&jcT$vgj?Z+b7Ux_9qp29y^w?2bPF zW^}!qfI;_b%!)SI#sJ@DY`hz{dxov59p25>SY_6|n})kgyy?EVU9mo-do_yY?M%0t zo?N-|>Awm)4ci8JHwK$YseLwc?*?BB4$Ef1>j5ga0zNNLT>w7?~4rr?f;z6L}BOnn1x*C9dvVs;J)#R8;_-vMO4pVb4tL zn4^l)`6|^MT}zq=EeU>#s#z-O?VG;+%d+j@EXqi_xfgfl)@x<$tC=#tEzq7!?*=~2 z@bhlQ(!1F=oOh%7I1(hF)%tdlvf6I22>m}3_8aUP^K{Dndrke%!E>#7b|Wko>nvS( znXV0znQHO1&#=|j+^;mCUUEMq8wZeB19Uloo>Cy`0#fUN-ZH@PD&X7%xLyOwwgB#JfM+}4eI2NL1Ms~G zRKErI-vMfN0D)aV-R}DfrQqKCxu$^$FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x zoF_o>hT;vy8;Um+Z+LNYHX)mkO~@u>6S4`}gls}KA)Am*$R=bHj8{^b`yKIIqTX@cOt!Z({OD6%ptX#l7pt( z4I6FfPNfC(@GCcp%8p8&NR zE-d20A}%Z%XA6sXS!rHYI_H;_=0V7VkOv{BJaEbbn~+V&CS((`3E6~f!kjl@8tsM) z?I+GcdukK1;IB!M&dd`5Cqfpr8xFY1Y`9N>Xn*QX~5Q107{Rq8#XQ13jC8qz6cC1$w=J z<8{FK2H<)VD0>TVzXN!70N!0dY7pUF`_}>L;_5*NfC(@GCcp$} z-O#$Bbwlfh)(tOg&L(6NvI*IQY(h36n~+V&CS((`3E6~f!g1DwX{;L|3og^=GW|UI zDm)Q5^lBmvy~?6>GY>Y2Dw>K^QTwbawysu1$zNdNtdUgFvL2}43{+JBoicV!!=9Pg zF-H}p^HtIPh$>D$rizBg)jz<{oI-r}fx^ck&HpeAam$v7qh@5@jNROs)tdWaRfUzq zZpNW@lcQlb(Z~a~_N_COri!(_S=VmvHteP(9#V$XZmRYS8g^sVv>T@}bL|G-3ku8U zK&%;PYXRa>pyLFPXa%}X13fVyc@9Xm0ln>j;~T*F9pL&NDEk3$UjRIp0Pkg>@(SSl z5vaZj_^$yqH-Ny;K;17u@K+%8;!G$uF9D4&1CdvN=Cwd{9niV~h;0PgUIpTtfR5K@ z7D|aNGr6LH2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6ft)8m?S>1BxUh%|i^kc) zB3@IP*OborHKlnF@*w0v$Po`5@xUfz6S4`}gls}KA)7GgO_)Zzp>aavgzA-n#>qr7 zPWWr$*UX}JBmaRlnvp6*`OuOEYBz)K7mS*Pvs4%}7HyXp4bq1{YT8k$F--QascVetZq3ZSbJ z=&1sdJ|Oir&|3{Sb^*@afNL*MwhwT>3wZVe-UC48`+)BQp!!3=|1nT=5C|Ls>J9_J zBS7d=Abb>P{0xX31Dd}8qF(~7$AQ>MpzRb8KLd1}1rq0huCIWeuYu$@&~8%SQM(a2 zUsAyjzyz286JP>NfC(@GCcp%k025#WOn?b60Vco%sNGPzp>{*;~=hy;q_3juN z^w~B-k@c;l6zE&S_?B<}j1OtBnUy?jjo3bjtz?sx=k={cS*>p;DXZ-Ui_rfw;c% z^^ZA4`@edNF{*Yi-6-Y;JgC3 zegw*{0`6;o=LX>Y8L0dP@cjx@zo@{oc?qa_83?=r)U5@A>wwS(AiNQ1d=-dn0-9d~ zqFaF0Z9r^0(DphIe*@@v6G*%TbiD)g>;RIxfYffFcQ4?m1)Kq;P;$Mea9INrU;<2l z2`~XBzyz286JP>NfC(@GCcp%k025#WIZuG@4ObU&brDw=jkDE7ys|W}ES>W!OY(273)K~E~7w~ zd3d_j^yJEwPycmVPI>tq-i<*i|IcstZmc$|(&99x^=|OB;IO<8xIO^NJ_Ot!1D=C` z_YhEd81Nkdsy_w%M}eBpfWR@J?h7FJB@j9egiivEr+~;Ap!qBiJrA^g1;oAv+P(qe z-vJ%p1BoAit_wiVB_MejNL>MXe*_#!zNfC(@GCcp%k025#WOn?b60Vco%m;k*SdN=fL=-tq};q}eggls}KA)Am*$R=bH zvI*IQY(h36n~+U7&YCcdcaw+f?c12C6E6 zPFWQ#)39eIcFa*l>3o%Hj;D>(H-Rv9AyBVE%H}02a*{Z&TcT-h=KI`61{lz@%OkJ$MFOb*=biE7o><5wufYkdy?+1Y6BfuE~ zTn#{37;t|Acp3q36Hplee4hi=&G2sgE%a^#7j-ZJCcp%k025#WOn?b60Vco%m;e)C z0!)AjFoB#WK<|bti@36gD~rb2$|7D?nwOQ%`DLYf5b_}8LC7f&obtdXWD~Lp*@SFD zHX)lZ=S`T#yWvXviL=t4-h?dpYtp0>JP~jrWYN1R;EbCQ&G&(LGw7B-CLT@(({4s) z)U4K=8PmPoRPE)oZZO5hs62qdGG_Q3+RPfWEl-)HZ_UbFeJd+tIqjx)IO}F~rrr3y zoMo$hW46-t;@elVZryYz(pxtT$M(-0(zXFcF811ie_pBt#&4EQ}j%~l}b1?nnh7fQj( z**RZOzz@I#m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28wHslW3RHP-Eh85 zX0v??MEg^Ne49c4TsAuT4MxgjpxoTMKVvQyBe<>Bo=oK?Bjx6PUA?g=ecMUObbVTl zq#Ty?I|ijzlXVO>gJQ(ih^;|?!1&t(##;{#h-wXQ6n?Ud_AoLCp-T^f30wTMC=Dk34 zAJFx24OGVf|2d$h4G6RYb#Wm0EfDGe!ks{40?JL~B9$9Xx?uuLfC(@GCcp%k025#W zOn?b60Vco%m;e)C0=Z9s$_*D4aZwQ$6^*k+MZBOiFDRY!3rh1K88K|lNI%VvdhCMT}V~#3H=c}Uo5mlUiOcf1}tABurIfeM{#oBi1?vWs%^bNh4 zq^!0ZEJFX!g#8A)#yp*J|6Wu7bMRbip4|vf#yU$^W~Sq=eX1pVX|}E9!vdx0@Xv2& z-MqQmu$z*2NEzC^2@D!`W3gG4@}?1(H~3z#SS|yRD?sy)K=dlmdJTx(0NQ>A;=ceL zzXFLD3t-v21oXTNBwqniYk}T%fTIj>mIE#~P_`LxdjQW?!0QDnD*#_5P+bN1eL&6I zK%g3^+X)2yKxhvTt^pcrfk*&oeh-M&0j>1~g;Fe7!0U)I0Vco%m;e)C0!)AjFaajO z1egF5U;<2l2`~XBkn;p+-q5_Ec|-Gt<_#}w&L(6NvI*IQY(h36n~+V&CS((`3E6~f z!g1DwY0Mk5EGS-3ypSkf@I;UWPlSncB4p9Lk^jIZ%}6bxe3(i7+RdGJ3Pw%DSt>*Z z18;^GWrj6r(mr9?UG$ozWA5C*@+s8bRE+VhPzZfsjZkELD=7u~)}TC~Z;iLMCPf=t zW47ffv-GW5nX7N*TZ3-KX5h`R+D)HslWpxL`eA{seeHuv)7#&Fly&W<^FsQ8H&uJq z3_b9s%2$CV|Lx zpg9FZZvw5oK&<3Jd_D&dUjuYFA1ss-r4QzOK>NfC(@GCcp%k025#WOn?b6 z0Vco%m;e)C0@QA}u!swbxUguPEiB@7rFmWHoL^U(2O$qa9)uk8z%dVOLN+0rkWI)Y zWD~LpbKZn$v>Pt8pEwKcsZGd&za~YxJe~+R5wfV=EZ~rvj0gNIz<{5A;RBPG?lf-( z?pc;q3kPqPfq64Vvu74)?#qGn3V^ym29#yIx}zLYag_ApMrez^6?X6Cf_t9j}9&0v>X^D-%Npg z(|%$E@(sQhM3!|xVgt~%5$JgpNNxgBuK~SV0EZWFRsgO_psWgT`vA||fVUc`+zI&n zK=mHLUjx+C0)YTf_Z|?e148vcI0!U;1VlnWa{~|!1FfF`u|}Y+35Z94j?aNaGtkun z^hAN=3CK68R?0VmD?69~6JP>NfC(@GCcp%k025#WOn?b60Vco%m_W`GpnOC5hVl*N z8_GAlzB!waO~@u>6S4`}gls}KA)Am*$R=bHvI)mo6Q+@G46@*AeXiCQ)K+*R$bu)r z#5obNDBt8mC{e}f$5heqxGHu&p^Eaws<^&H741){qNzv~wa=;)bovAxS+uN|`t_TE zcQBP^pm=lV{d-h#3WPBO_vV%l(=W!1_MJkCH{){ZO~9ZFHfCM0G1%@DZ$@Y8P0y(Z zZ6(!zQkptK7qc$jG~8wCP4~_1iuEC@K?u8IDV_iDbgSvfl`EgN%S%=)U$OGJ6-u|y zWVP#$Pr>7M*&;OY#y)uH%@n2OOQSVkf8u2AiDo=eC;|@4Da~Oy1EkIZz2^Z(JK&51 zu5W>|4#3?BcoKm3B2d`{__~4W9>D(-P?H1#*MYhe5WES5dVz4spWxUyfXEu4*$G5T zfmRm~TMx9A0r7I6!wn=h16>}VXDg8O0;vk1w-Ru?2{_;SQ=#N~=TA9bQNRzt1egF5 zU;<2l2`~XBzyz286JP>NfC(@GCcp%k0KFToEaJ){t}Gg7D~ot-X+6>%eI5FC^P9s)hrn3 zKVx)Tt-X;|9Wei9v^5|6{ncap9s9y86apTL85V}DZzZKb-x@NOZf5AuwPqiV<5Qwg&wH<8KcbZ#`hd?3!Vw-R#SabmOT0ldb$GNH?GQ z*G#g`(EXLEng}z=Dm`JB7AfV62kl*GmzHXK=jqR#qdj-Fb@HQgUsyGFVbL?Iii-Z~ z&eUJbv(D6?mai!>Q!Jgb4Q-{AtUEb-q|%K+Y5BX`rJLU-;G~mo@U;znXpllD| zt^qu?fHweCz6bc~fa-d{9|USX0s=NL<40x_Uy7B%<>4w8@m;e)C z0!)AjFaajO1egF5U;<2l2`~XBzyz2;?h~MNL+OUn4W%1OH@vVpn~+V&CS((`3E6~f zLN+0rkWI)YWD~Lp$5|7mk!~beaNdoHtMsX>@I;UWPXyzHIT5lb-4t-Z&4`Bkz`7Z9 z%O4X7CxcNpgJqli;pCq?NAiqx&pcrm1Jh>ETj(Qp27lSnc8z{t;!W*v;?47LdnB7h z>6>~pNm*?-ScLwc3HuFpjd?od{=KID=is^4Ji8J8i*=T+!c5mxn5h=`Pk*vig&$Iy zc6utZF5Yw}(u+3@$M(-0QoJcIH{CAT7;Gk`)+yb-b13r$-wPH?67XIJDpP>(CQ#iA z_)8vwW#a$>Yk)c@5G(~kE+D)fXed_d3J zK(ZQ0?F4%LfMYM<+y}Vc1M}fu@K%^CDJ`F@;K6S4`}gls}KA)Am*$R=bHvI)mo6Q)sageVmp;q7d2_d6Hzo0qGNgG^>lrlc z#*8UH-5U*=54*wlg2mznf}4Sm2MBKk8ofZI0%)!TqE$ev4~V@Dv{eJ~oj`{lNbCW+ zYJi?vAQ=Es?*YAafa3$e`61x?7$`dkxDNrI!+`e)Q28m~I|@{P2KbKwHD3UMFM+z_ zK=33GIt7H!0F7s1-bBvRyb)a1!33B96JP>NfC(@GCcp%k025#WOn?b60Vco%a-IOq z8?G(l+9Iwk8fR;Zcui?uQ#$9@l;%OmgOCRyM?7%E1DlXd$R=bHvI*IQY{Hy3VH)#> zYwah_T6>xkvf!^tm(IWw0VhHh%^Ud-Y|@O>BFcxE)UVy#d8c61M4Y94@b8Ibp<|<8}o%tr~+D+$$^aF3I_N*Ct;7!A} zLE4SQW>s2Fj6l1=_kzOG4n*QW^S3~>18D68VhNz_A`tHaI=X>G576}!(31p`*MU?D z=)DOzeg&K_&V^$05>WOs;C==0tOdO5fXWSkZzE9sD&XG))Vu}+wg7e8fZ%o@^g0lJ z1895`h`a?fzXL>f0Ij>`7D}<*b926+fFFPfFaajO1egF5U;<2l2`~XBzyz286JP>N zfC(@GYB$twsNGPzp?1TIo3jbogls}KA)Am*$R=bHvI*IQY(h36n{b>pVH)kmC=0IB z=Q@28g%zF%vfzm@aZZFRYBviwrV^C_fR~l>v#fYsDTZ8_9@wW$zw;mXYdGlxePnzNCXn)eIJczAilad3$z7*_I6nnkM}e}>0QWJ#^9A7j5~w^5_)Y@VrvU#Mpyn(PI1ki) z1q8naLf-)4?|{bdfyfU)^93M!3240x#I68sKLYWqK*u$hH;EfGZ#eLV2`~XBzyz28 z6JP>NfC(@GCcp%k025#WOn?dGJ^`9HTwBDoMO<4n&ej(3y3)L^bk46U&4Z8!ArC^1 zdEl4_HX)mkO~@u>6S4`}ggI}*H0BN0+E1Ld_B1DC!C#Xuox~FXCqfp@n|#b7I^88)g8*RU2v)poUCMm1!28+=D zGhx5Mt}#!i+`re<{~SEmnrAm!O6OZ=Sq(zir{k`DswJM9YwK?M8>MN(yPK@m8Tvz0 z@zBGQtkM&9X^~RCc+lR3c4?`$cb@*-Iofk)TPHs{_k~q+7ZyFUs;I~=FIlmC#meVa zC=GX+deeP#yJCGvf4!o;e!A842M4(~0`5&i#US_Qx1DzVFHONfC(@GCcp%k025#WOn?b60Vco%a-IO? z8_GA7Zz$hTzTqX#*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL?|djeL_Q3$E7ZYJD@c z6`lyP;E6DCPJ}GVH-(x{qKf*>Kvf0MDXXGo8urY@jyb9*ov+f*(cPqZaFdFKP=y$% z-Q0P9S++%-MH!hlqdt9Rx#rp6yWhIuQ;osMn{;pb*U!-ZVX}K?`RzaPBB7ZqW}8rC zeJd#i`qpr3)Xm68-pm-qkvIE>({4s*b&pK0gV)8X7 zW{Ra#wxO+*l65C%53Ake>o1(Fz0eGqMkvCXX4h@XjW&F!d@=NuCQV<70IAP`-e$n@ zCEz>`xK09PrvUdEz;hPxo(C$w0(@Ts)!zXA?|_=`fxr(y-31_c2?$*V!dHOCAA!hK zp!pgQy#ciT48(o`+I|J%FFp*t<|QEUGSKx3(6bgut^-mVfZmOO!won$KU^reJP+r5 zF#$gS6JP>NfC(@GCcp%k025#WOn?b60Vco%m;e)C0#t6esECV-xTt8HEh^$=rFmKD zoL^R&2O$qa9)z6oz$p)GLN+0rkWI)YWD~LpbKZn$lp8LxpE!%`sZ7X%za}L*!4m-| zLKc;q0?xP@(R`mf-H!a*{ftpka59*7GwLSFpzmJzn|w29|ET^`e^bmB8q-_hJv%qk zbw|gd^lc|8qv_n#=Ek{kY$*-MQ&bq<3x_j_sd0q;u2lGg;HvG#N`b z_*M{DwgIl~K-uen`whVJCg6PwsC)NfC(@GCcp%k025#WOn?b60Vco%a-IOC8%j5nZYbSQ zy5Tj=*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL?|djdWv@1y|{FmA-+x3Qq)C@I;t6 zCqfpb8xFR~Y^YCxXn$&Gn`GJO$T#`In1OIJrVa{}7icI$r!g9j{!WFP8Dl8i?8%LA zpD9QHDIg(V~r3YV671&L<}`Ztx-b^SWA#vB2)|*Yt&c+)EYHn z#QJ@n19l;Vl1kciyU&^UIN>B|l4s6&`p^4uUa1Fx^g}@AVPN19AiD|3JqqNv07F{= zTMJ-s1sq<$xgBu%0QU~S;|E%H0p0-MYy0PVoj>@`6<pJY2g&lJg(KJsHgEuSU_^pbFEKvRd#^u!GyRQ+3 z;oD;M{z$`R$|d0_qqJ@ot3HfztZ#RyZ!yx>AB#u-POY2k$I-f}(3l(Vo`1Hq-TNuK=Na zK=@T4x*zC#4T!xCbRPubhk)M0K;lgxc?3wk1*DGwnYV#~cYy2(Aonhie-9XXAFzE0 z*wcWcA8=*>*J;2#0C)z0mMq}?0`TPk|5+fA2ZHB-_90j|p~ipVf(9nQ1egF5U;<2l z2`~XBzyz286JP>NfC(@GCcp$Lo&c>IS~s+AXx-4d;Wf?Kgls}KA)Am*$R=bHvI*IQ zY(h36n~+U7!J4p$byF<~F4N~SeI0!jo(PiQi7`a+0?3fO!ZjYYSlkW@y1kbF~~P)VvSLrr){g`+1j>Ro~vyoc|wUd zf5C^+SxoY3Z97F?ZPlBE_MZ{^^;VTt8l~aA#^LASx#lXX0mEa=v$Z)gGt@aU(@nv9 z|HTq{6XH$wkYlQOruNWuJap3(vu=@9w^(joQm}WSRkuvtTcVPDwIH#q1w!`$ z;roH;13>3`Aod{8{SXj;80dWjNNfUPJKzb{Xc?=kM9LPQaWVT!1prX4+DV+5R3xt2Y^r~5bgq^F`)B}f2r5Sx+&g> ziZ80*2VeqBfC(@GCcp%k025#WOn?b60Vco%m;e)C0u*n!w1`WKxU^`3EiK|zrFm89 zieFWl2O$qa9)uk7z#$K8LN+0rkWI)YWD~LpE8c`f#2YTPpFB(LDNaa&za~|>DxL^9 z5y~ju%;$)klE?eZ$9SJLLU;a#pW$-%=Hfle%j)6a?MiTOO0sM2P;D7gS#FEfzeDk6 z;)*wag;=Aaa3+O%u#NP&b-vJUQfaJSC>OCO+K9KnU82AXtehlP30rIDS zp)-JO5U^(f#}|Mz2e{4x?mXZ*2eb?U-o`peHa5V&4hYzRU=z^p074spuoH+j1D!4) zwh`!d1M$s3ucxkFm)KUvH4RLF2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bgJOPR~ z6mKZrP`sgd!>gOK3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g@kTERuGHsBeL-b~ zCxRq+B21nWp^V~94P+8U9KTf&kp+s_u}BfkOB8W-sUrGrQ$$yTBHHd$DCeA)b0pET z;evNFEa05KgPAlJOE;I_zeWwGR_IIcZhnYOQ?I%+=4-goyP0s_O;E4-HPx~@*k=5^ zoA_gOmcCQ)Zk~TKRCezsa)p66gXgx&<{xVVb7Xa3&J45hPb*jc`EQeQB3lZ)8?$Aa zJouEMeK`R9P4J%Ftpuy+8C5a4_n zaD@SP1n@+GmIHvd6YzBb{umH=0|<5l?L9y!4up>a(O#hQI1oz!-6w%~66oy%5-A}0 zA&^P~>3(=OnGC%f4!mIkOn?b60Vco%m;e)C0!)AjFaajO1egF5U;>p-fZh#P7I9?} zR~AjMl|{U+G_NaN@#{+SAml;FgOFn$IOc&($R=bHvI*IQY(h3+#hb8*cf*zTlV_zp zy$MP1*Q7~j* z48Hgixoguy@v?h29jB|zvoyV?26~NknyFv1ps?h|^~Y`;op&SP-9(P)%~tKNO;i7x zWD*v=Jb3-3`}Ek?yaW=j1YM@H7D}4#2ws@Hqj0GZ1hA!Hqz>8whO%!X6;H4d`qEVy!^87l>~MdVN4*2axmw zsa-%i0A$*Lfgq559>}%-q+XZr_({bt<;f4g1egF5U;<2l2`~XBzyz286JP>NfC(@G zCcp%k0L2@MHxzFu-cY>Z#m(7-Y(h36n~+V&CS((`3E6~fLN+0rkWDzjny`p?qmu+z z>T{*OfyxR`1WE8jm^>#!8O57g4!IfCfFGDQ1-F`U5phZwcQXoOrWChk+10F?!FkuJbb>jfc?IC^*YD>^SGda2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6 zfr=+U>xS!!xUPuniYC~)B3@ORSCy{#Ri$|l@*w0v$RQ6L^1vo!6S4`}gls}KA)Bz` zO<2Ub;X3=tv(BE@ge3TD(xa>9iGUNKjMfe3+mtrjr&jdcHnPW|`5?N2(Ki?W^M{SW zsS%*Dl?hmu<#Jy4iI#>1J$Z-PoGu zS(tL0xdwW6=?SYq3uA}2Sj%Ooqiy;3+N62@iw402qc~d zlI=jM14xH}%*(()7|2F|TolM30ERjN+d;s72yh$*oNofIBY^uYNH?Bilx{@D7gX>A zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@GN;i~lDBV!Hp>)H`nzISngls}KA)Am* z$R=bHvI*IQY(h36n{a|PVG-#@NP?^Mxk|r^x(ZJON$^CNJSRdKrJFj)B#P*2P(<6E zirBnb5smji#960PM9&7GVpJY2g&lJg(KJsHgEuSU_^pbFEKvRdM&{Jx zyRW%uzRTibF3QDrN4Y#4r6loY{B)Gns`_G7gz@4pJ3jYm-AvR`H?H1!me%b*mAg9P zTgqam(kM%B96rYok{6x#=drB_v0jsu=YZ4@kZ$}b6dM~b zunx%Dfm{=icK|~h0NX=={b9iI2;kfVxE=-ETL8~ipye^Z`#9ix;-~dG|C2xEiUuaY z1egF5U;<2l2`~XBzyz286JP>NfC(@GCcp$Lo&dEQE-d20A}%bNU<-?QNoihEy5g6V z=0V7VkOv_rJaEDTn~+V&CS((`3E6~f!iqOx5$%Qx?I+JddukJs;IB!MPR|noCqfyu z8|fcdqZzG2R0A#PV(q5je(ji9ILm~ggn2iRV8-=UPwhs&!CY;swiqsKjq*J0-&OK# zZCfqR)wYs+y|x{nVK;488+NlAA4X>}$>+CI4%m(X_O}7YJAm^9;CdHuzXy2U2UX953iAn+Uz z+yk`l1wtNfC(@GCcp%k z025#WOn~wYmltt)5tkQDu;oR(v@|a*UGYmx^C09w$b*oR9ysZNO~@u>6S4`}gls}K zVa1!Uh>~3D5ph4i-;!zPJ}YbH#HDS6mk4kMMM@TV#gvyG%r!a*`Y4T@;HQ=y=9UeJ+5&j#JFesl31Or{%PgP zKU<}xEACja@~#zfb8~@sW3rg#o-?BmZ}7b!u|$E`0ie4Rh<5?KF(B~NfC(@GCcp$Lo&d!giZ>K*DBe)K;T6u=gls}KA)Am*$R=bHvI*IQY(h36 zn~+U7!J4p$cvFQd<#VMzSL#z);fWv#9Z!Twb0U;cys1}R5=C@u1X{NN{gNVjuEU;L z*fB>DP4g7GIhvPL2QO)jKt#pen_&-UdATK=#U)ucSGQ~GR8PiS##@d0*C^ghIPqrp z)x?{znRpX-{@l`c65`Fv_kLY=@uuT+m3fxt*VMqTu}(AfOBNKC+_?VOjiVE9%$8|# zThLH!x=pctkXW3m#L^5TTtIRoka7d*%|ONj3~U3kEkLdn$a{gI?SSoRz`hf3JOenN z1zfuU_j7<}574p~@V)@}UIhFv0fAS5;69-JRUot<2)_nIUk5r50NP(_&Orqs!wTL%5Vkh>l`)BI27nc73s)fqHX%(_KZ-D0_UNx|NQR^2jnZ=Lqs zIqGw7G*7){?%k{AE^N4cRYSwy#?}mGifcEK!+U0psNF=i6wJF3&~763%jVq}<;edj zuHE2!L19S(nLc131!O-2a%mvn4-91h+Zn+ADd6}FaDEQBz69J~0iLgcmTv&>w}9_E zz<=*ApxCShg7*RK_XD8^fbe=C`XJEx5DNfC(@GCcp%k025#WOrYWkP`jaa zL+ysz4YeCy+ni0vCS((`3E6~fLN+0rkWI)YWD~Lp*@P3U35#eqMoDm;KG*5%DXj2B z;M}XpH14O2+6@QXls4R_R`lIg;M)}ZbNSflH#I_Ef^u{1{)|{7MsZuMKKYuI8%2G& zaa6a}>hrJZO@F}4>xF8uR9l2BXOPanXfeu|n^Ro5!S{l`vJ1!ufT1?Pwg<591spE`&KCjKOMv?oz_Sl%c@^;P z2Yjyq{?~!PK_GYtXg>^u-UPx&faqI5=P@AmHqiYJ5I+I*z6&JY1Cs9psSkkkM?mId zVBixVdkV;%0rH;$L!SY*F93TE%8lbJl^agFVFFBm2`~XBzyz286JP>NfC(@GCcp%k z025#Wl}~`m4Hp%0Q4tpvO|V5pyr48MC|&UjO7kG(LCAxU^Bp+fflbIJWD~Lp*@SFD zHetn^u!wR)+l00WwJQT{lgVV8@YlqzSw`ii4l0Qvx*8PGcBdjXuU16kJmuOY`ghDtEniAX0Yo=1RkE8dG6; zWb-CyEEH;ta^G#EFmLd^V6l7yIKBm(-vO?B{}qzoDskk1>6VzwO;4x{MU+K#*-g_2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?b60h%{7Z)o1oyrFr+3!AeE*@SFDHX)mkO~@u>6S4`}gls}KA)9c5HDM9+ zrdkpdFDPEYHkOWVJ$#Wu<(Y%phlR&MRqll(?iWt0E5yx*;L}Y;?b}Ujv^Abgz zU8>N|(afY8m`N9#HwE`=$4taoCPWG5%~*|rS$+k*X4xob)fCleM!S>R%~c29TvfYS zJto?X>-B%Nv~K#P+!bj%UUu!K|5WjTH?6zYjXdzCai6hJv8k3LUR~kN5hmW?i@{+z z2zU+wEr$W`n}F{K;C~AU90P)H1MTksp%Xy(T_E}%(D^NfC(@GCQ$JN=-qH-5my#*Wzhs% zS;PxV^TN^%csB+A)O3}y&BMXlm7v}jN2{^?u;r8 z6wvuJ5ZeiKKLf;{1$uV_iRXai9w4Nfc6dT8`?LtZ)o4}3g>J>HX)mkO~@u>6S4`}gls}KA)Am* z$R?a%O<2Uf(PLpO=Y4RwKD`y52$G<;A|}m=P)7TvhLdhepYkJ#o(=G6pxqSSFqfu4 zzA5ob}q8c{E%$u>9dlUTNmzKzLH_KhNrl#rUnP$BZ*6F6Ec{fcl>lRsci{<7e1$!4- zb<5Pfb=q_1sL#F8JoT2jcdweeu;KPq4GmUl>54m6th{T59J#{Wo56EiW%G}<*UReb zXPAwDTDkJie|zDe_cHUwB)9F=k7Di(z8EZ)kATp}K=>0NdJ5<~1H?WBx<3Qrp98&L z0*SAH6V7UKeY-nTr~j025#WOn?b60Vco%m;e)C z0!)AjFaajO1egF5sCWW2Z@9LIYm2zHXo9UR;^n1zdFhH@UYZ9X4?-S$Ywc-HVDQ7FF(>$I^4FBa4>u#rXx`MTGKnHOHUh2N zfPP64J=bB+EbN%0h^Bc8Zgy8jyNNz`v!&-dXg42x{&?B7n~u{}=2@CuQvd{ z`NP1_n}972*pC8^Uch-Aa3uiuNx+i?TKWKQ3h;dh_|rh39|&fE_R~OU00?6amoAzG zI==v7IiUM25YGd>=b+srhN#_e;0+UC0!)AjFaajO1egF5U;<2l2`~XBzyz286R3Ow z)NZKVP`jaaL+yqaH)j*F3E6~fLN+0rkWI)YWD~Lp*@SFDHsJ(o!Xny@?pjuMQ*q#w z6xt!QL)ak`aAsH;wVPVFB#LNWqKLCg714K_BDxwB(RQaIHm_Dh<2~?j)cyDH^j4?-eX2JTPQ7WnntG#7!qHhw^7+X(Q{>fFy-8^Q z8L?k)RavD`8s2Lheh!{%uCf|nz?f%iGRzEBhM8`Pe|NK`@9=!N>%e<2lwG|U%oJB| zB8T_P8d1GzZZ4R3W426_2g8O@OuWGtgT`|2d}uanf#iKa>V6>o0FYS^3_J*A9|Cd@ z1NldQp-q5oGhp`sj%|Rm1#qN zfC(@GCcp%k025#WRByPrh>MH3xM+ecF5-oyd12{_Us#$4ArC?xgq-uhIS*_?HX)mk zO~@u>6S4^_-h@Te8za>TsuNTvCZIaOk0c2L&&?lvZkAEK;fR~k#{1MlODb?~3jVo# zY!sXtp({bV8Og9&uKF?4%6nU;{y{o76VAEWbv5VaylYc#=O%S{zGZOhujH<$4!v1+ z=cegQap$Jz#Oc|ib8ZBjo5)MLLg%JhZa$@gtsHjsY@7&-yil7PJraHIg|hkz>$xcdQ52531Acn1LAAmGmefiHky4ro6M zgz`Z691tA>Ivallzs3f1uLI(CptlJ~IDq5^Ams$o%|ONl3~U6lZXmb$SM|EQ=T}_L zzyz286JP>NfC(@GCcp%k025#WOn?b60Vco%m_Wr7pmRg#hRzM08#*_ z6S4`}gls}KA)Am*$R=bHvI!?x6Bco91V`3zWDUJ5x*b2*?O=2hGrBT5H+39wQ}TG9 zdeJd!jnEC>7OVFQr{7$>r(7D2Qj&Kw3aw_os?3OMzo8&TrQXa}J)P=nzv2JuYlMEB zZ}mbITBwLDkbO7ivE)^uSzMV@JvFjht< z@7HN}^m3c}O4au;U~9luuRUP+{sF^<2MltR_JC17IaPawQ8sC>Fv^PdK$Tqb#GCVz zZ^||CCcE`lmc|V18~Y0%m)*X((!`s_R9GI-zG-tCFLi1JvW?VTD&JIFs^zxK733Ry zFNiFU0r|&)p(g;_cEIif96JD~A8_pg+yTJT2DAhL@AH7K9q@Mmfe;XU8E6j!p$HI; z0?`9NXD1Nr0=i>B{0*SD8%Xp3$vBWY3Z#31%yD2K0c1}Cxg?P91BOz7?IXbcG2|P^ zCzNkQ#TQoa126$5zyz286JP>NfC(@GCcp%k025#WOn?b60m?UAUc}`^TwXN6mKX85 z(!8#8#jh*PgOCRy4?>Q4;Ft$CA)Am*$R=bHvI*IQ6>q{K@=Z18YH+Rw1uP0!6tE_l zMN~%lMuJeHh}JoZXqu;p!J8Fv{8mLo7ARuJB1JSWQN-D$3I!ca(5VrdR}Y&vm*1%! zGZ$x>(3ddu=2~Q$A0o0qM!9y4X0(5y{_j(}nF%$f+A=}3o3R;sW6S)?(%kWHa@WV+ z@5-*-^q(p|^rm&!x{-(8M79)YH+qXvZvB5pq21tnL18%!I0gXcAmGXZ?k@mO4rnAm#+Rn}N6s=-mh;+(2?Okn#ZOZ9t|4 z7-$8uULdy}$oqhy9f0i_!2T@Y*bO+J16+Fm_uhZ2*Lhy}H?C-40!)AjFaajO1egF5 zU;<2l2`~XBzyz286JP>NpyCNoyP z6S4`}gcGa@i)c57Nw(OW4hSZig}ZstQJQAFcCis)abQ$)`OpkpJ@x((=; zuT*7Moq48NFNAfv zscGI#Q_Q+WR^4K`c}cjOqZvf$LAld_T#(~&Tpt~1{9|w99K;k5jOaiGsAe{m-9|8kuAlnb*GC=+`Ff;(z zJ_GEZ1CB2N=U0I1Yry>t;Q1D4`3~^jdka3_+FR;%{`+pJ_@V-S04Bf$m;e)C0!)Aj zFaajO1egF5U;<2l2`~XBzyzq?aB&eA7jbdX1Y2Ch3rq9D(iOk3G!H@^gggj2=Yew` z*o16CHX)mkO~@u>6IQ$li>NnxdJ6Ou=qX^)I!^?i2$ShVD5H8)!znkVPxz5kHEGyT zx%dud(p;?FTz>x=MVwk8N-%G(?#h(n(-b_Xc0|3VR-HgI-nYL~`R4j@9D5VgYm!a1 z{1IOPD12GTKy$y)B0KKh1!V4s~11TSn z-T`F%z`!mb8vt@`Kt2czJrCGk0PHUUj+X%ED}ZYs;C>bG><3z21H7*TzJq}O5D+*F z1m6VOk3hZ&y+!#(aAgM*U;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qq0+eqk-%!4x zd_(z$*EnYrvI*IQY(h36n~+V&CS((`3E6~fLN?(9Yr-P(O%(^la8L}r6M83;$vfe% ziC?pf@=ZNQ-IO}!r%voxv_=R;h>NwG%kM8Qw}`X2B=cs>4VXJr$;LE%Tddw6^EWMc z3u-r4oq4nSYTC`%%)AL6y~Pso{#x#Gq&v&5-E^F;GSAZNni^H|nP%#jEEvosH?BW+ zpNg&n-bfxvR?qX9FRW?4CMjaw}AaSz;W-dq1dbiT=xO)`vK1bK+Af-`yk+Z2=G4)1Reo` zn}GI5fzTEpycLK(26R6D>v~=6iCNfC(@GCcp%k025#WOn?b60Vco% zm;e)C0@QA}u!swbxUgt~EiB@trFm)TieFlq2O$qa9)z6qz)25mLN+0rkWI)YWD~Lp zE8c`fv>P3#eMrK{b^5d}cp~sbm`o008MT{QxFm{bUZRMzOBK;~nPWKe-pyd9xOWpdyl2*k-p!!5VBC$_VwQXC zhEaGo_+oHawgWLA(7gkQ`+?qFKq3Gn+kjLMNIwr`+JS)%AR7X5F9Z27Fcbl7`vLoF zfa7(*c@S_N0^ElI&znHY5y1Nv;5!ER-v$Ej0KpSL`@2BsJs|u(5d8q?{0NAB40L}2 z#7_aeXW-o=KBafVaW_nW2`~XBzyz286JP>NfC(@GCcp%k025#WOrY`!(7T~`L+^&( z4ZRy)+MG?uCS((`3E6~fLN+0rkWI)YWD~Lp*@P3U35$3)R89<3PNYs9$T>eaX7tQ%9cWnwuuyDH<{#0P(E=?nk6-1WvQxw1PqO=pTbH$5j#&mOIF zGfi$gG#cjyUkv_A7D#*nBy&LOERfCvnRCFv5Rh&Bclb3nAioY6vIDjU0Q-8t@gU%Q z2yi_NxE}#Ln}C)_0q+*Tw-xX|1_T}lf=>YLPXeK*fbi2mbSKdH3=n%3=-v&)p96aL z0ExXo@&zFEB9MOR-|KamSN@%s5oH2QfC(@GCcp%k025#WOn?b60Vco%m;e)C0!*Od z3DCLWsv@o`;;NzvwyKDil;$O+D}G679)vsyc@T2K11CJN3E6~fLN+0rkWI)YtauX^ zac+bpIPRvJi}a~n@I;UVPlT#Tb0U<{xvArPo04by)Wb_!BMig0#p?aS(Ki?GDVK($ zjMBSVtQs-X#`<=L`WEzVCY*Oup;)VHb<>y% z%OiR>ZEmC4s{Q?G`1@ALBrMb9$p0xEbz_qIp3-0XM7>3i4+n)MtST%KAQJ@!4glFs zAlC)tW5CcGfbC7degts51vrlZuD1dAJAmf|(DE+eeGl-x5BNU-0v`dvkAe12fY2!* zd6S4^>SQ8e}Zn)5X@+`EcHX#ZAniT2uJP~jrlu^5pIO3+{ z@jf+T^Xg&e=JGqWV`kwj6S@+{-CWVLsa5@$k@TP+{AjKEXRm19)T$4e#^P41{#pGv z-|B@rN2b~$WJ%lVL9#MS{ z1GWZi_1Xi5?;kK+c)%c6X%86XlT)=<7-f_83ZtxO4^+ulm2dun7w9Y|`TTZ@yxOWa z3GF{4_Uo-Gt29c(dyT`-!E?=3Rs&QP^K8v(nW0)O(@oh8zp*r)fPCZJ>qZ`W)7*TSe502eZ@E;yF(Hgd#}(uod@qPBP9Wb547mW?CcyqE;Mf8 zAh{n%y#}OT2QmkNfkQy{Fpzr_$R7cQ-U4jBfc-e+8%Kijjo``-Ccp%k025#WOn?b6 z0Vco%m;e)C0!)AjFaah|@dPN}aCs4z7jb#f1Y2IjOH1?8(iOk7G!H@^gggj2>4B3T z*o16CHX)mkO~@u>6IQ$li^w+yNpQ73SL++8t?)#U1W$y?b0U;czL^i9L=lbmD58Iz zP7ysDfR2qo>o%ZY!mjJEXBKvpEU74y^l#1=gEy}cy7M>8(YoBfxp>d=vbs2UyAu4H zk_?+WRAZ(*=E`FA?~I9FGs2&(r+HJddGl9I&8dc~qslzh@{cs*Mv_-wFmKLp%Vpl! zPW;Bwyz^GM>(j>YxS2PR+`*B}o7P7P%p1MMDEIw;qcCsqyT^ zfahbNL?K=qn)nH4yy<==>IleFt>kdn+uPwLtHEK;nKN z`2dhw52POiG7kX*4+GgpfZQe^|0poD1+Z-c>@9$!6>xe1*LJ||yR}~D*>P*d*A(yr zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@GCP4Fs<_*monm06Wc!_g1A)Am*$R=bH zvI*IQY(h36n~+V&CS(&%uqG^G-f*q`cF5Nle z%o}_$SS-%~o@arU-GKKwz_$nR?*#%c0Kpf5_LqRrD?oT35PcQs+z-TF1G--a;s=4= zLqOs%kbDzJ9RbpB0hwdKz}rCf9UylC$iE8=y$9G*fc-NfC(@GCcp%k025#WOn?b60Vco%DxUz&8?G(l+9Iwk znqX^-cxh=~TDszwmgYgogOCRyCp~b|1DlXd$R=bHvI*IQY{H5+VG;ADN)lYE&!zg+ zbXIsGU}OfJl}U3Vl+nDY=a`#PNBq=5Ov11jMO9Ub`#0zJFE6)+gUK=_XgAkl&s=yd zPLMUCy7b$8^~p4EN-}S*hXhk?G0Eq*Q{@Z&V{khYwTUkvHD2Znd<%_8)TB zbH~q=-Mr~IU1gr72{ttlY^>8v{gMT9x#Y(6$8H>*d87Uct&RFom^b)huvoqZ{NDh9 zZ-L-NM_0n27B5WWwH-Vby>0L0b<-46oshk)LPfy5&~aubkx6i9CYGFyRx$AIkP zK<)`3|0FQ<6kyu{*!_TG7vKy4t~S6O1U%0JE$x7}1Mr0a|I0ui30AW)#XyDRRxSt7$dM$6dOC=2Ne}Y2i;W)_X7)2&zi8MaYu2)ycKm zRxjV6ZLdD@W<0%{wySwJ=Y5-UdpE(?{=*V^%e_A=9!u#GhLlRGsUc1WYsN} zo0k;qU1-%UQ}@_iZ5Y4v;(nq}~P6?*W_x)lOY_3g6~C}F4?-SD1q!_!&C973eYcIQcPt+p|E5NW5}cc{9XnH}$}Z-DTaCJBY~T0tzfbMvs^e~U zT}`_gn{hYMM}BMRc^}%%=er*(yLQubrnq*~bK>;u(P=kk%QU&~0mCT9-QbHsVc7)4 z9tFC$0P(Fr?_)sXaUl5wka`kGKLuo-1_pKl*=K;B83J4{ z1MV>3i2yB8zQ@on}l|g=%aQc zxTb>%FaajO1egF5U;<2l2`~XBzyz286JP>NfC*GQ0ctnYZm8W*yP6S4`}gls}KA)9c5HDM9$hO63t;Hq{C0g_;JwAKtf5hmt|P)6;h4myb< zx*8PGcBdjXuU16kJy3Dh>EPmQ06I1Tt=jo^`_8^ zDbQrf_|Mg{%o$+dQy}{pkoz3Se+dkI1=#X{ z{T$#J0-TNYaBOUVdmZ4h11(K}*8%u80DdPBXa<5VpnW3{as%PbK-2?tZUbU1KzAz; z_X54!frJl8?f_DLAib-;UY7~fSA0bQKL8V80!)AjFaajO1egF5U;<2l2`~XBzyz28 z6JP@LZn(0DD~q_YXo9UQ;#H-2Rq2XfRhkDO4?-S<9P+>+4{SmLK*q{%ZApA91Yi0CqB#yZ$dBjf*yrg02=JGqWW2WIO6NVB- z-oSzx*IPZk8~FxvwW-=-xUewLDkbO7ivE)^uSzMV@JvOgf8B-mlZ{ z=;b!`m3oU_HehSORjahDMG+g>b^~Y#l0)=I_s<1o_9);u0604VR~O)p0iHL2mTth?1Nh>A|0odX1%k(c_5=_*351hC zv=8V^0kIE(?lchZ2YNF=;xv#P08)cMItygJ00weE_AHRgL%YeJqjtk_H%x#DFaajO z1egF5U;<2l2`~XBzyz286JP>Npz;Y&yP6S4`}gcGa@i)c4APH3D^y{e*mB?+Dglj=k$qjoc&BW_9_?=xQv-n>Q_&ficg zmzy^i?^#|}3kPpkx_Kk1Ld@UB@{H;I`RW^)&fg^U&NxlonXkUlNVA5<;?a7c*ZdhF zOWIZ^*J@jR$^Oku?fK*E-~1JCs<%|b*in_9YWYXl>MWA{7j0|0u$>~Ww(3nn`_G8S z^;VTt8l~aA#^LASx#lXX0XB!Q|@wq{^7X!H<8@Ik^P&V zZI}5sdO5P-Qvb$mG0Q#nEBH6~UT|5y1M>I&CtRDgfXxorn*fIcaBcuxPQcv^cw9is zM!@R^e47Ek2MBBff-OLMD-iMm;q5@w2XyWLVt$}|7Z48sy=_1u2qd2eQtd#x1IUDc zftP`77|2C{d=wZu0N7p!><9m|UgtRUpS+?d6JP>NfC(@GCcp%k025#WOn?b60Vco% zm;e)C0u@hy{tZ_cafK0A7)`JhM!dQ-uP$Bjt4s4B0qrH7KI(PKA!nc}GVQJsWhx2F}HIFqmd!%jFtvoLZqPLBA=Y+$_7A1#~T%%5qhJ z87JRn_>Jl*-e?nXjPl8;npb0#O`2C@lojoPDmkcHLOO^xs$f%0yjiV1Zj#S$%O&2} z-uO>Tv*&km*H_7NWfyNER~URVcy6m~{;}r&$nbw|nqfBnY30g4TcxEd?pU$%t`%}~ zv++{NMsG37t!s6c?i|s)!54$Y(hWFz0B0O<9R=LIfaf^Sk^sCX0bdgE_W^+v5d08m zPXnQTAe;fBr-9A^AT|heXMy+^KyMC6oCT73AaxE%4*{9R-@&r60oipx&JN_8fFTE9 zdl0Zc1UMcBoR0vmO@RB+-_`3pTYgvZH3j?tOn?b60Vco%m;e)C0!)AjFaajO1egF5 zU;<2l3DCTuc|-Gt<_*moUgexk$R=bHvI*IQY(h36n~+V&CS((`3E6}btO<*lH#%1QcFa*k(>#T8jwUA6K}=dB z5K(dYW>~~oUTzC#aY@?E)u+JJsiKUrjJF!~uhG0QRa*=fM&B6ad5~JFEmiVtZCfqR z)wYs+y|x{nkvF@qX5M^1@}|7zjmz^pOKU&Oo8z&KR`X2FhMBI~FjLIBMONKnxp_&! z-i21(GIejA_S`w@b8j?Hy=Cs*tL83jxP4Va!`}p&j?-1vp_G5tKZNT{s;5q@g z-vvDH0WI$X-VXrZM=)>vAJe=MT-Lz^m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyvCu z0L>e&E#lfDt}U8iYm0bkXlXO zxo?Ah6ccao#o(}H0RL$qFaQJxf%YsA`T_{&faqDEGY`bh0o_ADyz#%_*w}!?Iv{BW zQcXbG0c17+15O~@4CGuuej_mC25egZ`&Pj57~p&ya6JLIp9DNl0WD7h-kpH&8NmN6 z5ZDa_p99+W{8zm$wD-R%zM_C1fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)AjFadfu z^ls?g(7T~`!>gOK3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g?}jVwC(lZIdJ~f1 zuSt`xnkNEIgfee{2z?38&9w_OrZr*|x7F&C>D){> z=Vn)BoSR_#e_0~y{#)+ycn_4_xoJ96+_~vFaeDUXoEwwHEJu!v#<{^4gTK-NghD|0 zWgr>`IwL?V3UnU;;+;Tm7m$bn$v1#hH<0cDGI3zwD3I+1a>s#u0vI|8*xm!|?*onx z0Ov=5>tn$E3E(*ew44FFp8~$m0RQJe;7cI*70~`Q5c&oPe+xvv13K^h?|NNq?SFGQ z0~25ZOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-6SfX)q96>(J&R~1dLRYkm{G%qP# z@k>hcAml;FgOC#*IN^a!$R=bHvI*IQY(h3+#hb8*b0Z|dMfzN%UqxGmCxRq+B21nW zp^VN=9b6JcbTufV?M_8(Uag46d*I=$(hayz2>tMFv3kF7^v%V4%BA5bC3!caP-d>`*VL5e)YOf;cLUR> zUVYQQ{R-oAzh0;|OSMJFlD5^!wc1uM-=J*`7q&)up7!r5dA7E#mgj0)Nxoj&nl5ao z$TQ87NoUc?`*qqKz1*h0Qg6}A25b%3>a_<9-#=iu@PI+C(jG9%C#Pz!Fv=$F6-HUn z9;lM9I_~B#c!3U6U(S!cnIf;Y>P89wq|F-nRq27G+;K{P9H&+^W)0hg&Bda&Tf^j!y%QU&wZ5YM48+NfC(@GCcp%k025#WOn~YQ)f=ieRBx!>@Y3dNLN+0r zkWI)YWD~Lp*@SFDHX)mkO~@vkU`<#=z2Rc}$+Os=>VzctYf`4u^F+XjP)7Ae`Uh5N zMynCkKusD!^r#&(6K9zaC5*fotLZb#Rn=wM<+o+(&aryaEB$49H`ngnocC?Y?cK!V z|841OT_AUz{QC2(F-Jx1E~a%J_%%!z(606O#!(NfqWVm>IZD60Q(uh@hRZ^3~+r8 zxW5EEUjZ#&1Kw`{-?xDOJ0NiH0ys8nf%f}=(EUL80U){_=zI`}Jp^<=48$J+dN%=y zM}g!PAhi`pKenJ=mw9{vmozW|Ccp%k025#WOn?b60Vco%m;e)C0!)AjFaah|@dW7I zaAgr!7I9_K1Y23eD@*gr(iOk5G!H@^gggj2=z)VC*o16CHX)mkO~@u>6IQ$li+DE% zNpPV)7wQ{ntnfsT1W$y?b0U<{yP3~1HzkkwnU4`aYlJv|L#K@3;NYRjKN*00f*@K3YG+JX2FyrbA`9X^L65$f{c`H!msJyU?mzrtYoNo;yc< z?v3WDx6HkJ)!c;*x36kwuu4l;+_7TiT`S}(2{@75!I1@=p1nqMjrRAas#;C8Nnm`D zxOD8zG&nhZN3WpY;A=r;@dBCcz<>|P?f`OrAioP33IMj}0Q(-mu@`W@0JvTR+%Ey1 zSAdp%fcI6vw;%Ao1_WLQf(L>2LqO;-5PlPg9sxSv0%FI2?ze&XJ3#LVAn`7cd=E&y z52QZ;G9Lj09|PG>fZQqQH~BNvZ#edb2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bg zJ^|`C)NiQYP`{ym!%LjA3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEf;C|g{e}zfC(nX= z>JyUSuSua!#}ff3LK*d&8Ym@-IDV@lA`29;W04}7mnh=wQbqLLriiWvMYP?iP|-QB z=t!bxgKk(nx%dtS(~K;GT%(LrD-0#rHy4bUi$Bg~)h8~?O& z<)8m1>>{1$M12e|M3A4oQ9 zftLFK@BM)90l>c=2s{V`9|GDR211Vj;Y~pFQJ`}R5ZelLKL*4f2YR0X5>Eojr-0Pc zKzb*Tc?KAG7Rc@fa?b(zJ;2ajz}5lSL;q8+bG-aN6~CesSaM!8evd`VPups?%%9Ipb- z{ebH=!2LSlIS8~I0=$O--vbN- z@42FZ2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6fr=+U?S|S7wHs6Hc%uETY|Tq5b4pXisfI68tqO(g~gjI1$RI-PCf% z&8X)4z`H59)r^aVQ^K?xSTL8|YSe!k$MjaG{(Y)9C8;;(2i=U%z?-(KsW<8%937_b zoF9BMMP6;yn}qhC5&QL4l~o$0;l0M;=is^KDysn^i+Q%@w9HVQmgy$f`rlhx-+_A5 zx8tp{t2cw0;_6N0@Sa&CsyEHe#!D?5BPIYvj*dpX!54$Z@*v=O2xxg2@IC_gHUa)e zfxs3ZxD{xB3VZ@9RKi;K9pXo4*+;)SJoVd;urSegeR4?-S< zob$jr4{SmR@OC1^Jz88*vRKc>38w`J-dq;oUjoSR)&b8gPNHsyA1yzl(p z()RQp+9nrpbLr4Q8t*5lvGiA_;?W z^m6kO_$%+I{>ll!|1J=C4+y>ww0{7EJ_5oY1JO@_&Qn0_4AA{45dRG5{TxVq2_(M) zQeOkNfC(@GCcp%k025#WOn?b60Vco%m;e)~cmi~8 z=-kk`p>sp$hL<*H6S4`}gls}KA)Am*$R=bHvI*IQY(h5S1Z%<~&J9=DPo7owbS5Oh zUy~M{fhPh^gfcoeb#O@((bb@cwmTKEd9@-M?}3N2PN#^T4M4|6pmiJ2FJaep*fR?| z<|v|Ro+1WsR>bjJ6%kpW`~!^5smFI;BXq;J#p?aS={FbeDVK($l;quvLaUjtDl?{Q zze(yJp?Wjn)SC*8y9qx12TSBjs5jomwKL5#HHl?9B$k_|n01S+y2a2~3id9v>XxZ{ z>$K<2QJ;IGdFm~5?_M=`VZ-gK8XBz9(iL~CSb5h9`AXw%8dG6;MD?c4T`=y(WHHN) z$Npz;Y&z2V{_E-vEY zq6xORh*y>7Ri!I_RcRiCJP3IZa>xURJg^Dbgls}KA)Am*$R@0K6Bbc#swKg-`dq88 zqqM>kK@vO>CeMjbM)gMe2UclDs}a>eO&a!YF27SdW+u)up)X6eDl& z#o(}f1;oAvy1xP9-vYhg0f~G62*+kEkh%{@-w$LS00!0r*$08#LqPsvVCWIR<_7GW z0fz^0ZUbB`fV&m&c!8GffY%53b^v}q5ZDC-13-Hl5DEg}=YeQD(AfdRLO}P+Ks*fe zM*di@OGN)z@r!x#126$5zyz286JP>NfC(@GCcp%k025#WOn?b60VY82hTaXm8+teN zZg_cfHX)mkO~@u>6S4`}gls}KA)Am*$R=bHPOv5{;@xni{p49`Pj5mJ{55IPRq;f? ziBLxGWE}Hy7_&URDhUZ&!kPGirCHsBW{kWTEDcarbUu z`qZm$df`_Xw>$MhZ?1-7BVf~B&tCw%kwuTE^qdZUhca=O_+g8hSwXGyyuWd~i zwo~MpX33%+H$WzSU)B0h-$eZtXTD9Rj(=dF;D&f~K%7gz8G?x9U#_}4FcpXR{1X729^kE?LCNOXW z$i4;Sjsf|%fuVN*+eyHl1RQ;UGX=Ol1l(!B(+{*{0PktQHvsqtfj|}regU-SfY4bW zoCl)kfX*Qx)~GtXQK(8H0GyzEmklFyGol3nf)2wh+0~25ZOn?b60Vco%m;e)C z0!)AjFaajO1egF5U;-6Sfa(nw7jbbB7Z**i#YMcbG_NdO@heO7Aml;FgOGzBIOu^* z$R=bHvI*IQY(h3+#hb8*dZU*F*XnbvzM!ejKl<>Y|;@{M&Kc(?Xw_0I@z4G^>-~?4)NC)NSFWBm_EmUyS{W^hZW1cqQ^Ce=9fZo(L z#ayGkW~%zpt4#uOmAG^sjuHM%>oy%^mWNfDyzz|iA>%?sGK z0}dbH+yS`!fO{9<2>>l^fHw&Eo(KHxK%fH%hJf~$flwF-M}TM)=sW<#I)UykARYsH z-vAQbK(Yr&#ewuuAkzyB90#%qAa@c1PCiKiM^t=q1wQ~2U;<2l2`~XBzyz286JP>N zfC(@GCcp%k0282qLji{Z4h0+vIK0j|n~+V&CS((`3E6~fLN+0rkWI)YWD~LpCs-2} z5pcNVe)252r$8YI{+d+kj64x=B9u|UsaKs6MRaTgu*zH!{gNVjuEU;L*fB>DP4g5w zI+~wU2R{iOilQnfBdH(D%Pr$9F3G;Rq9b!9cV@Kv@3{1fFIKNfC(@G zCcp%k025#WOn?bgJON5KTvo(oMO;=i!Il;A;?lghbj2?&&4Z8!ArC^%df==FHX)mk zO~@u>6S4`}gcWbXBGQdc5?rOvRr&_%Dm)P+!4qNfoCsxiyt3M3Hxv*uD4 z<#HjXED}x$GjB!`fiC}4*S@Eo){T6FX3-cfY>o0f?cY`MY;9XD&(*e)Jf7A~TV<>p zTVSE3`5dg9`?u!HZru!Kid#34!+U0pXx$8YjYVhJl&;<2dqH7o0~|rX`8?oi2izTi zCj_*-40ywUF9P_ZK;Qrn>;&4ofKUtwzX3$MfzBQv76-bI0`Xp;_c)M90Lha;DhZ_f zfJ_P)_z=jZfm}b3&j3TG0o$j5{WHMvIpF*faD4^1zlL_>`G(pJXWcLXCcp%k025#W zOn?b60Vco%m;e)C0!)AjFoDV^K<$Rw4YeC;H`H!;U2`@en~+V&CS((`3E6~fLN+0r zkWI)YWD`!XCM=@eaH0L=S!hpfLK6HnDbiK*M8JtqM(u{fZAu&LQ!DyzyIi@sc+c{& z(Qj&mp#`&=;>u0WiPN)3tK3w}jqmG9S8njVpsx%8p2q)$USk8i>j0k}@HYVg2N2uNfC(@GCcp%k z025#WOn?b60Vco%sN8T-5f>G4QPBijRKyEP^McY9zo0Y^LLP)X2sz(@^BvfPY(h36 zn~+V&CS(&gog#WR0392F)@?w)gk9HR&n)biqll(?iWt0E5yx*;L}Y>T4=^#O9^ZY9Fbv-o ztM|nSw{lrH$|$v)#R3XW^;qBTP~T#tuRj)#QoE@#OD3JgD4(1Pl}C>|CTtDZD%t~8 za)l<{cw_%-Y1_O=?mC#=S$6H_N|SCHQ(<{T?WWCLFzKe+Vv_rw(wDB?;Cn$~c?0lw z1A!hO7zf&q0-;_Yd>n`-fXj`yr4>1Id0Ml>ySHfy@9fFbHI`K<*15 zp96-@0=91e`?rAOJHUDGA}BU%0r!1?=YF8&0l>Q+@I46l9|8go1Hngt_DzfGb)iQW zaYX|YU;<2l2`~XBzyz286JP>NfC(@GCcp%k025#W6;FWL4YeC;H`H#Z-SDdBY(h36 zn~+V&CS((`3E6~fLN+0rkWI)YoM25@M7!Za`^mG=p4x;Y_-j(6)AK~YiBLxEM*0WV zXhy3L)j&%cc5W`eQ#)oB&N87ZVcN|VJ)2t9k1>tKtycZBG;d7R7K40)Hqgc>&(pS5 z@@#EeEzi}ql02RRZ?0CZ3Q~LKx{kE?E~UFfL=e4*aaj5K&lN$2Z7A2rQ;S zv0cr_aYk*pU1dLF=&vU?vA+#x^NpznxhsQ}zlAh%0 z>C=DSkMlYN_>KVnqd?#|5c~uPodDWS!n_H8M)O7#eN_cN025#WOn?b60Vco%m;e)C z0!)AjFaajO1egF5pn1c!MO<6NwM8YiwutwY=6$7$eqU)`guDoO5pv7}$2_nJ*@SFD zHX)mkO~@uJdJ|?bZ*)e%rTSc|ucx!ZKLn%TAHt944HEuxxC=z%s!0G+`4#DThrW zi^C7dqV+*pY@I2Ky4kY0I7b%U56hyxN*2wJ$+UB>+BrtivGSUDGicykeSoPn*IPF? zKE6#4r$T6R5N~cpr>RurnQ^zVQM@TR@y4fB1sj8;jJ7G9coTj7AxpRIpQQHFZ=B7$ zc++~5kvIL9)=B1{sgrjkOy0S7f?5C2!iA4mjdK>vU$F4;1ybwU4Dm)N;>}vE*{Z&2 zoHDM*Xc88!)G*mRK|S^p<(SE&x9FvY^%__#uPYXdABeR8T>&8eHqa9U60JaA2uQvQ zq}qUVJ1`IiZ2JKFM}T8LQ2R09JP5cB0q!F}{ZYVk9BBLm@SXsgP6EEq0RQJe;0qvl z3J9G6+P(zBXMxB$AbK9?ya2?$2D-ih;+KJ*D==>oOaH0pYYO-Qm;e)C0!)AjFaajO z1egF5U;<2l2`~XBzyz286QFs+wMAT8#I;2wwzi13mgcRci+*cqUWB{|c@c8b11CMO z3E6~fLN+0rkWI)YEP4}WF>lI@f=l(eRNp{ng?|VbnL%geNArh}NAspqkx69HvKqiD zb6NBnWzlg5j!eXX$+D=KCR5H)#iXeala>grEVLQgO~#M&@@?VF%1OJq9ed{5dyR^M zGXj%l`bb_p#Bat-G6b9ZHeGqKzQ9|h(8FUfScGI$_Zq1}-D{=0)xGZ8UN22ke=d_I zse6MoMco^vyVSi&x>Mbc!OupM#-fq-Xw(NwwI-qd)8n|-s<2F@aqy^q z@HKd?xy-7Ap<X0 zI>V}&s~nxGzIL+m+I!4n@1OF-qA4?~9$i#b^_@f0a=OesQGHpta<*}%UZVw_B~`CD zc+ar%%?+MS|6SjAZmH`Pn&{qQ_9YAU&kgf#=>Hym+!2Ueocmb$g z3pm#St``CK2B3Z;;MoK;ZU($tfTouL-z$LsRUoh(2)+h{b^vW}0O2=*$Xh^kC(!v0 z5ZeWG?FQm|fS&h&#QQ+sULg4aklOc8m74TNbZ-O~b}#`Zzyz286JP>NfC(@GCcp%k z025#WOn?b6fubis_lE8b-5a_$bZ>Zvb2cHHkWI)YWD~Lp*@SFDHX)mkO~@u>6P8#L zW^r%0+Me18{VPH3#3=ZO@T2-e$fJ8x!AUnGn(}j_AW~is7bl0AHzSZ{hVWnppPD zkzKoK-M3@nklIc4E`6qAqnA4Vka=tf?FQcq3QH779{>hA0oxJ4eiU#V2Wme7oF@R+ zNx=OXQ2#mL`2uJ>1$fT@OL3xLfH*w_EOQsb!qdC?aX@B=UbCcp%k025#W zOn?b60Vco%m;e)C0!)AjFaah&?S>1BxUh%|i%M)^5$`O`J4+Y+&eFUHc@gp=3E6~fLN+0rkWEwb=YeGqHjq2t^S7ZDP&^{}68N4@nvC%`*Z+8Hqj1WF$U(Uo!MvEEzeUz(r)G0q%9WurqkG;Pe?P68 zVTavp8_v4z2G*H#KLnTQ?m?Pfr@wy2%(~W00CRXhxvj z;EO?F*$6l`0kxX}=N7>AGT?p%sDBmkYzG=&1H3zcrZ)iJn}Gifa6o3wg+(b z0NfC(@GCcp%Wp8&NRYB$twsNGPz z;hoLdgls}KA)Am*$R=bHvI*IQY(h36n~+UdVojJuyWy&K?X9h9rx0KijE>ftj(-TH z`G=54?Pe-;5?QoY$)fo&S*%$si|W5a#aX72MaN2@Wi`;S9_TgV&>c835eFvAqGp;b z`tOs);Rj^V`k?#|Fg2$V-+gcx3?9tNX&y`-OdL5U^#&G9fwslUi43wZ+~~!O_K_c` zcXJm8%or>`>S;I5Ge5U9?EGh`z5mmWyn8n{n|4zj4@yINHx1kL=I?no1`E8K#7Mjw zd@(pIUjpv4K>azua~^2C0C>L!n!W*imjVA3Ah7hG;n+M0gq{N0o(96pfyfFV`V7$d zED(DR=voEDp9gwg01|6~zI8zIMIf~SNN)rNHUYLqz`hl5c!An&fU^m3H3M!RQ2#pM z@dJ%5|6Hl@2L73M5@iBRfC(@GCcp%k025#WOn?b60Vco%m;e)C0!*Oj3DCRY$|9~T z;>w~DTUo@rO7pJLMZc>wFG5~~ya+kufkPhHgls}KA)Am*$R=bH7QG3xcsB;lhcODe z1Deuw2dG4G;tjo?Tl?G0qjzKc2li-2>JXK~OS;~>$#`5bY8uX5q03?94J?>~Zk6-Mi^Mm3`h#!?tBZ z&%5dO+~D1qqz0dM1oLk2#o(~K19*1D(|5O^O5?gc_00B!q#@JB#oKM?&G z=sXC-4gp<9fcR0M=Qxn~1n4^fBu@gV&w%vjz`z%P?KEKT104N8Z4z*P1-Mdx`yx=E z20WL5#sR=v{R@0P8{k_8`0c-_)C6jNQS=oB`~XaV2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?c{yP6S4`}geBI5S-cx6 zCsa=8UX{_kG7A17{HXpA^61@6=Zu@2=le{@e4iykcXdOpWOz3jKhDdmhJ(}PuHL|D zTp~<`*`_Ne(iV8*)SEK1(WJ5HrDJ1Z_Gt0IguM=XS$(2RYS!X_2E$$croO-+m8mZ< zNIzAN8>PRhdy}+S-H(wLTeT*k{?p@qTC2h`mBzuN`oY)WwdOLb&QvqaJjtvT!a4yD zt>aC;<-f4B9)f-2d%Jp~`A+qz@p$UqF=ow7t7eu|H#_6#46A0Ya&)Tt+R4gm?=g?P zf65b!rp%~%bWv56)i`It`~?diUm)GgzG+SE9ooL>Sf63vm@HNfC(@GCcp%k023&B0<>?qzKH9KxW1^w))(>S(!9BJ z(Qhuzi;x!~FG9|G;H(EWA)Am*$R=bHvI*IQMQ_3^_KlWrGw9Qj~GtC>5!BSdtZ!py5s(GUrHd+`sndZ$^ z<0jwcP3X`sEFGJFDYdWKx+d@DP3ukO-t=EuCz*eynmdwW?o2T2A6mHZk?)k9y1ESW z#%viUb??-TVD1gR7%Y|}K=>#SISxcW0Xk0rv6Dd8XF&XOpyvx9aSG@=10=r$QfGnm zIbh&CV7my|(}3d=P&)uPtA7d0#s;{T0rhskQv)NfC(@G zCP4Fs<_*monm06Wc#m^7A)Am*$R=bHvI*IQY(h36n~+V&CS((qSQBP3Z#eNo&>kpR zdw^0Dr6@{KMxnojKZHD*HmQJ8)zo4osFs%`};Ejw&Wig_yKN zAfoK@&7g=gFW(lvRScMwg9o0fv#78_^UwAb|CQ@(6<9fz5%4(1k!H-13Lj*5U{rbju24$ zF5qkfTRuyNsC%t+x4JK_i8sx|c{igo@g{!mmzMtB zv!wQeSKiONchjH9?%lNR+c9xS@1}LF{`-oJuxO>`d$TGwSr5WtIj1-*=YhlppzmuS z`3;b|45Y6B150PYu~`P#?SP{OsC59&m4K@jaMuC#PQbGoXmkPIH9(Ua@T~{@^+2Ej z2zr3fOF&yA5Z($zyg+mt(AfmUnt?7K5Pu!$@dJq#pf3O<-v&~_S(Tb}>nz?(lnF2a zCcp%k025#WOn?b60Vco%m;e)C0!)AjFoB{cK<|d$4ZRzBH}r0JcXKu&n~+V&CS((` z3E6~fLN+0rkWI)YWD}NH6K3&lZe^AN-2u7-g>eU{L>YySLvQE?-ohV39=#imxXEq2 zPlf1yI77I}_;KFo_&4Q3n}c)n!xLz3!GMAOatrUe&9~gXb91Na=9r|r)cqLjjV6sn zBkj?s51@fe!!ug=AUf=I*lX1%bl*OqyY_@mDpQ}(OUK5l@6byo^&NUiR-Y)7wiU;@ zN$#FysjizXwfj$fl6U8(=1g|yrsL@8Nh5P^v>0Ji_h?qVChI}?E4vkcWe<>k4;XkK zu!RBphkzpj)OG;QDBwB(xI2OR!+<9SG#&%IT|iSe;EMzPPk}%W5bOm)383vX5bgsa z{XjGcbbbZIQb5;5Af5(#E&+)Fps#v1KA#OpEd$baV4w!DtpM!L%&ycpo}FFv)dc(i zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l3DCLWsv@o`;;Nz&TUEqcO7oV|MZcvq zFG5~~ya+krffF9sgls}KA)Am*$R=bH7QG3xI5&a<4FwtsH1t~NwcLW&Qp%T29-W)1 za7kp*UL}j>$7Hc)u`H_p4i9IUMiw0_ftJ-k!+N0Ch(mYa$V42NEQ^|Hvgp507Ka~@ zMeBp|KfuhKN__Y0r@PEjCbwjczPbKLzBC*;C+}utV`r}aA~!yU>644AGPOt+OfH`- zzpwdWsW-!pyZI|z9gW2#T^)NfMp|sunuPjKkKXC9 z9G$AZcCzx?d(30+pYp__DKn}bT~t+NHO^Tuf5F1X7f8LQvd_F}*tTrwnK#wDGQ=AL zjH8bJ?2gS%#2b7wNG#oeI}X%;3V3>e#$Ldi0Gdt%zCOU;4+N4x@GBsc0@^MD;WQAr z1Vjgb&gx%5vatbO%Ye8Y=&1n`4xn!(kgNq#bwJt)46FuhF97znfMXp{`y$}n0Jt^+ z?oB}bX27!rXngrsl^X9Wzbg8Y0)7A{zyz286JP>NfC(@GCcp%k025#WOn?b60Vco% zDBf^s5tkNmX;FzSE#keUd2i{W-&>j&AumE+gdFw2Q4efFHX)mkO~@u>6S4`5-h^4i z8(m>mfg1%+q~TlDq5P0mApreny@5+SZ`_@C>ba+CGJyu5li zI9(3zO%^@oMq%dmk4#sT8`IUzs63#4V=`ED(%l#(W3cF@Y3jaAnxyUx(iC-Xl!~I? z{0%49S`1Pd_B!l;ioM2SltyRnjc41hEX`@?H%FSb6imNqP3;|8zv*}*L%%UWziDmJ zW;JcH9)!xWO;K5z0BDf#d-o)d{2z0|PO@b{w#O0ys_pwI>1RXMpQ-!2JbKe+uxN0UEyqyk~)? zbAazW;J*OmORU@R9FE=Jb1RP;bmq>yx^y zmQB`!&{)!n#&QV=3;@CEe}QIW1KO4WVLK410iq6|b0rX~1-j~hxD)7E4J2GZ-x?t4 z22$&RbUiT80N6GG_RWA}3sCzq;Cuyey$ZOu1NE-~o*h8r8-Vvspy@5Zw-fNc0|a&f z!QDV;5774Bzf@|%@Ba&LD#`?y025#WOn?b60Vco%m;e)C0!)AjFaajO1eie46QFv- z#YJ3P#KlD=wz!BlnC1uYUiU?ya*tS?X7f0Td+{l~l#qn-J?f+uwaQ#|pUuQp* zckia+WTl#A_HctY9r-0zoKxjG8wgL!014Nz$qR#=HtA1Uni9P@8qOU06 z2VeqBfC(@GCcp%k025#WOn?b60Vco%m;e)C0!)D34ZRzBH}r1k-SGD2Y(h36n~+V& zCS((`3E6~fLN+0rkWI)YEU_lc;@xNpvk07S3*8nfCsa<@HU%4fl}GQUf@5w*G~x&1 zO~zJU5DzDZfj61TO}Qe*j2qQ$s`C5k-jtks(>$De^H+5yj!C*Y`(})^*s3)N^`G9P zv1qLd%TyW%kLm|sgV&nNtU4$#=1HmzGeNOo#+#z9Ut79Ag?n@9Y+v5poBl+0_oj8< zj)_CMH+6LxGjD|A-aM(v9B*TgS`XZCP%pvnfyd%fJeD;;%nfv{2jcZWPXmzf0DUh3 z$wnZx6-ax5fo*_oJ79keaO?nT-vFF%0q92UB`j=CqT~$xHpNDbZ-O~b}#`Zzyz286JP>N zfC(@GCcp%k025#WOn?b6fubis_lB#BxVng|i%M*D5pONcTT2)H*3!HPc@gp=3E6~fLN+0rkWEo%{)br5&7NbD!*_j>1LkttRcVKT;&%Ib<)vzwo>TfsTeFm zGOBxxRH5#*(%tG_cWtkirl~)dNt4vQL7JlOjnZA}-gIq0274pS7LBw=qdw3|&B{Bq z7OkYiUWdI_eM0x`6S`|p=%h0B3B7b|tojbUWK!Rumt^&cGHKgz(#=)bCg0Lc{L^1s z`hEW@wI4rjpJcvM^>N0-$GLZmSu@kBnI+ZD&Nw>5s+p@CovOZevhvz{%wzAL^2DMk zGpZh4R8?g)&RH;j!NSKENHu4&OE(=yPfr?Fy0NOif1Kh58BGG&_TTq?=f+?$NX?0x zIXC!X@K<_(L@&^n0FtMHR3DJ;2L_UW?L1(=064w|YQF)TmjTxmz`gWe;nzF~c%A|p zp9Z|kfu z1xUUMq_+QSr6&E_zZQKp0Y3l}U;<2l2`~XBzyz286JP>NfC(@GCcp%k025#WbZ+R} z(7B;=L+6I~HfIyE3E6~fLN+0rkWI)YWD~Lp*@SFDHerc1VHW2`6lMW9y%u^c)J>?H zux$!9_bQLh%~ZG~vS_c8Me}2_ShH9b)qjVFvrHq4j+H>mY5*f2=;n0XQ7Kv`E)kl+ zZI<#lbNtQqNAjiN$T@j8BhYH5E6R-a_BZ^WFTYw%F2AkvhpFD&2|dPODU5nkqclG9G^KPo+L1{?!rrD)8e^0&9TJ%!K{~d{XgD(b+<#izK2L@UI z+fKm#4&c}Y)b0kHdjQvafct%*elOtp0BGC?cs~M~_5;3;0slcDa0mz<0YXQCw&Otf z6CiQ|h@J#GKLcW)16^ML@l!z086fc`(03L{o&!?nf%FAn;A_Bk39t`9y>V3k8&@?j z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2`~XBQ1k?--f(dd7Z-7HQHd=s;$5YASLveP zRhkzeFG5~~9P+>+4{Sm6DG zjS)Ht+?vU!2)qacHvplHK-(rDycvjW0irJhov#3~SAnkWK>RhJX9tjY1L%7b zNWKN6b^_^lfPr0rEdNfC(@G zCcp%k025#WOn?b60Vco%DBe)Kp?E{_hT;wHZ_XxU6S4`}gls}KA)Am*$R=bHvI*IQ zY{C+2!YtyAt}v^>DYsB=p>sm#gl$u>*;jcKZ>Dq3P0llZrbA2`oD-vtd--nu&5a(; zyu5liI9(3z&8_$|OB8?Rd)us>jhgw&_?bd)KT!APu42r*`5R8Eg=$iUy$<`IVz03n zrO|P3oFDz0rQyZjNbS`dBL#DBT2p(6c5kX1Gu#_x@=)F75xF<`X7E_{1MZK3`h$Sy z5YTu8@E!%4jsw0=0RIUfa1sc9282Eb+P(n7r+~;AAo?ZHc@~JB1G>%w@e4rD*FfSM zpzkt}yaJ?_{sx}Slfb}JfUO3wI{?Q@ptcrp)&VXj;9d>Xy8zD`pwa!CN{x5@Z@8?1 z2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b6fubis_lB#BxVng|i%M*D5$`R{drKGn z-qO4Xc@gp=3E6~fLN+0rkWEyh@cTsFe8E|;*_nzh=_S~ zZ^}97CifvfM$xenx=ohpG8jv9qkVJzk=rzJDul>Ey%`k?rb3ZxOgGyql&41JyhHq2 z+Bd^azVT^Q%f=x6^(OX>a$Exm<*J185Wd51@F(iD9dncIn4=r5y zh}Ae}!TbdaA73C<@5-=mOct}$zgzqLaX3RyzQH$x$+7|PZUmY(0lv+Ee+v+J83?`t zgkA;OwgchUfXEIY`UcSXCJ=iI=-LUy-vN4d0g2r}-yR_O9*}w;Nbdy(J^*YHz}^8k zqCo8dz}X474g>BOP=5^YbODXsfHw{_eG2$`0Dmv+n?Qp04JY3)0Vco%m;e)C0!)Aj zFaajO1egF5U;<2l2{3`;CqVm#_6_YD+BdXsc$afFA)Am*$R=bHvI*IQY(h36n~+V& zCS((qSQBQkZ^{a@6r6?&4Ht?h6i?VT1)G4CNBgEy5lUpyvKnYu5A+&k(QyZkOvHi7 zvZ$FRQ_xWbovC8$%q2p|LSz+i1`p25w}~?=C-vs0g3QgtnUNl!uZTA0!EKiEnBI-a zV9`l;LuD~o^wKnSUnWgb_XcT-x;ILNoO-i;IPYe3rr!8Y{l?Pz)*Pulbn$ZDy_=TP zW#);hUQ@1EKI2TiMhn_Xs$OyMo{@PsW_ULpcHKz48+0vv0AS~uWa z54h?9cLPxG0X#1Ojg5eJE70TxeA@tj6A)+yf<7ShI?(2yQ>h8J%qjZ)Joy2b025#W zOn?b60Vco%m;e)C0!)AjFaajO1egF5pm)QSMO<0Ll|?1CvWPdA=FO#xesgJFguDoO z5pvc8XFae9*@SFDHX)mkO~@uJdJ|^xZZxzZXhYD3D3}ew6LHK|VI;&ndN&o2Nn}ws zTNW4R$fEmUS+rNlqWLjdtXV9J>c2z8Sq2-2a?Uj|3F1x0R$dSfCx@9g-zhPJZp^4Y zsHAvvO}Hs7@us;r;!WtSIhKy6e=D_b`t>bto${fX@2P3yiL6NeOU`aKx~Z^l{1N!)oyMX9!pmPrpdk^S(ABgV-dOiRW`+&ZWfaHE4^)Zk>2n-woY%#!o z3~+P-wcUU-4!AxA+&w^jFW^Z4ji&)`AJEhf_>zGCD6X{ zY(VF--&Sg3_TO?%0~25ZOn?b60Vco%m;e)C0!)AjFaajO1egF5U;;%?fZ`3s8;Um+ zZz$gI=H_fdHX)mkO~@u>6S4`}gls}KA)Am*$R;eYCd?w<=nJz7oN^207CI+%PS`dD z8+?^V@rDy_a+~f`0VyfNxXJi&-slK86S4`}ghg+{EY6Lf z2|*KrCPcwZ2%d=3wF)C3=Fz#C$^kby5BI4QEfbdr-QYG$d7L@>=K3S~(s1MvdN;Ea zBW9=%mX~MeE2sF*yP2gtHjVZ+Upa-oz+0uzn+@=5gk)6r8mU6vYjYlUbEo=xlXRE5 zAA`Noq_JqEJsS0aR%)JxXS5crq{Ci^y;glf_w5t9YftE;GW7|)bZo5p4!vYj-=UXe z^@%d6NW*U82Y+kn-!xZh|7_=bdG~H^HteQ49+ZakZkka5Z6&sVqB(=8O z%)7xCgTrzNNE`wBjsnT!K#i zQVnw}HEGXWE@@x_On?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_=n2rfp?5>?hTaXm z8{XHPO~@u>6S4`}gls}KA)Am*$R=bHvI*IQCDw#lyc3Rg!1+hno2A0sWcYE44@ zr^j)vRbiP*WG*L%7~cprsSr%mg*$bo3<^-^RC|Xp2|M* zreWK%p-0}-)#<;l*=RAzrmkA^{ewfhH~41oST+OcEx^FbfNdLKZvq_6K&=mOz7Dwj zfV%~#4*;IGfyN- zrwd4Q1ATEI`6-a<0n)v|KmxFx0qkGGy>XnSd&8MGOn?b60Vco%m;e)C0!)AjFaajO z1egF5U;<2__zBRx;p!r;F5>E<5?fuwTTAoS(nY_uG%rG4guDnj>4B3T*o16CHX)mk zO~@u>6BfM*v$!`p8WA)iXhamuh~SAhWUDYDVjkU_>F`NpQT=yW^e)rLqGKh{vKnYu z5A+&w=nfpAp942(I^3isLUVP)wSJ?1bN!Kdc~x<6x*YVIoD`b*iZGL(u`#oh-$VBX z<8XA>#^2~AWq6IjQYKAO_XcT-x;I`McT?c8H-E#)wH5;glr1mYV&KJzRX-(}N+P!I5mEqp#5k&vHBXV!>&ETX4An65C+kkWvFwhLxb^!J_0LPm^?OT9zC*XSLca<9VuHO~?j-LDgOn?b6 z0Vco%m;e)C0!)AjFaajO1egF5U;<2l3DCWvdqekz?hV}=-s7B2$R=bHvI*IQY(h36 zn~+V&CS((`3E6}t)`VHy8*O10g41uI-$Lz#+6mjHVDqo?=-!lb)=lnnevFEnG-%&k z{{WL|uJ>+ke0-ZG4hGERpx)eyP*bVsGltvPsNR&EdgIfoij6^1#@iH5y>Yhw&eBl( zd#SzV%E}7!o$5rK@tBBn?-;XYrd2abs+*m0bcR(kS2;RWeeGoBwfC6E-aqAuMN?)} zJ-Vo>%4(dmVE%%Ik1vo~Z!-3#|I#|i{4@3alJfouX8l787d~>`8`oy2HwMi6=~%8E z$=DlwGiWR!!2K>z-v)TvfyOZ4{Sau10KN{u9|ZyjfM6#OIt;YMfbcOO(gj4jfzCJ( z`xNNv0ph(tPXb7s2KxGdWIvEf0_m@SffQi-8nAx@I4%RVR{-bI-{bQ=3AmpE>YoNY z%YnufzpvDIpZPskH824tzyz286JP>NfC(@GCcp%k025#WOn?b60VYuN1gPF{aS<06 zadAz+n$;LN+0rkWI)YWD~Lpi{6A;)SEI25fmaQ zL=;Sj;E6bAt1up79@U#lPP)l;%Fk4>b>=v-g=H#@gGcp)ufc20WmX+T7xN_5d6|HR*6}7!?e8tk$DrPP zk$5TZ>P^e(GV?_BY~{+?#+iDJ7L4YSs#hGmXJqP)$zqn8BO_98@XesH)B#>6(6k!x zxd8tfAm9dq>w!=`(AEHiJwW6oAle9YZUtgqplcftZvuLnfrJm}dmTvnfm92S4gdph z1GZg&eK+9P1Ju3;INt|cdjaa?z^AXUvAMkz*G#vzdhXDT(s5gP5RBuGlmsRis zFaajO1egF5U;<2l2`~XBzyz286JP>NfC(@Gsy9?`sNPV$p?bqRoU;kpgls}KA)Am* z$R=bHvI*IQY(h36o3O;1FpGMlDa<->>Mhh;Xr0hHVcQgJ`c)p)n+o_OvZ$Lai;Ht) z(fzP2+N)&I{Fp4(ES5#}-(}IeOe53J`PR>Ydy}!17evL$VdhPq$Ir}F6c_Ul+dSpj zJfC#fUr6z$%xr`hqnD12g*T(c0~7W->}BQ?DSGV=!C3=&Hh5a@!1XNP zeh#Q#1$dqZ8eagsYk{V9fbT`XzX1qr1cIA@&}N`*%O5H=;g|owH4RLF2`~XBzyz28 z6JP>NfC(@GCcp%k025#WOn?a#Jpqb0Tw27XMO<1`VoQs7YiZtEy6Cr-=0(VhkQX5* zJ#f+kn~+V&CS((`3E6~f!lE}}7V$<;9fCRpb%=uL5IhlwY!${r%%gY%vBn6m<_=l3 zOq4~#WLeZqlSTi1vN-&JELtCwDdwnBQU#=>4C5x_$9bb8;FJrIgLE^LATxwJljBoU z6~$&;##XKzp?8BpIHfi4X4`Py&FBog33>lu>9`2*=9SQayn8n_XR>=Y9Y;@38ku(^ z;N4V*Zt!l*Qpe7WDL8t#Oc<&IhsCQnEZcx^6A)|sE0O)xeNCbht zRv;MyQttxkHejF~uzdj7_W_QNfZF|l^JBnu5O5y?>W=`Pqd?NfC&^o0eUy|Zs^_6yP6S4`}gls}KA)BzonlOuZ zBMP$!oNf!<7AhxHPS`dD8+?^V?`A58+~hpqrxI?`5+Me+S<2(g`8U@e$(M>F=j7gG ziZL0UOdfw+tSC1EpQ@1J&0=)`kE|#_`T}p2LXV*`28)o4>RuyNsC%t+x4JJa@uo-v zZ=x6fVCioAcd314)9Jj6H#Zx2QymXVLy9*I+w|sgvsMV}SX0e3tHC7jUr~I2+>O}+ z?Z%$r;tXmx_+C(0(m?DI&@}+WtN$H}jSWaF1N!VhvIa;wfb>dWpcb$_2iR8uj^}~e z7Xas4z_ku=zX;TC06ZIk#!Y~CGtjgJ@VyN9UjYKI0>SM-=ry2i2M~S(h`b3z-vT;! z0NfC(@GCcp%k025#W zOn}-A7Z!105f>Jf*uo;-RhoB|F8W=ic@gp=TT-XE~5q0rWisB%cLR&;7AdlV0^lE@@x_ zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_=n2rfp?5>?hTaXm8{XQSO~@u>6S4`} zgls}KA)Am*$R=bHvI*IQCDw#lyc=C%7J<`kq1!^`gvtrqreI^Q^61@6hf5-h>c7jP zcbP^O9V>yB)j-2~px1~)ci_lG9LU*Ek$w*%hSfTkUQ?+w8JCJ=ZF2<`+z?*MJPfbec0vImI12Xwv<#P$MR z9{}-vK+i`&Vn5LLF_1h6qz(b;Bf!8>z}5xWyP@AW;?!?M(U(^6126$5zyz286JP>N zfC(@GCcp%k025#WOn?b60qQqgV8jJRTwqjU3ygSoY2IDB=y#XqMaYYg7a@l|aM%Nz zkWI)YWD~Lp*@SGuqBmg{{YFbEf>H#fh=NHGJQ3$?6~;x(qkdBkr9>8oACN`sgR_-MSGPjnjibI>NN65zq?VmOh+&8W&a6+)APeFG6DbDOIi z(T~zLPx;MMZ*o#^t`58@jC$kKs!EMPRcy3SZZg%ItIAEj)f?N%KU(Tudsu4Ues)LR z)tlCv48G~Vv`#YrOf`TcWpK^}v;Lul3m^GzVoqIMhI*q%AkF(nq~740L1Xz0aC{Eb zegQa70j@KE`%9qyEZ{i@G@b{%7l5X(0pB-(|1uD`0tA;n49(_ApzSFj{4@|*4n$V~ zozDQVXMwKgfcPq)=XoIU0?@Y>NUj4?F9PWez`#bp_7Y%k1RPs|S})+-2Dq9YuGF}j zALgnCCcp%k025#WOn?b60Vco%m;e)C0!)AjFaajO1d5&j)f=ieRBx!>P`%+@&e?=) zLN+0rkWI)YWD~Lp*@SFDHX)mkO;}=0m_@xQE6h4@>Mhh;Xr0hHVcQgJ0#+W?n@SG5 z$#u}rRG3Li1m*!{w{HeboO$`yaAxJ?-3;&7OjS%7Lmszs<+oA2DLM6K`*7;b=uEwF zzV@)C;R@87y5k?@UA<{JU1pxB8aCz1jGb|&UZVxmxuoh92k#k~dZWx9s&i>aGW7=E z3>wQ0!2JeL|0dvh3uxR4c;5kzJZYfu z64V>-0M#2#yw{&>J1kcad8nB7nRuJ zBHmn@H5J4fLU_t~> z#4%fi@euQ<-c)eX&4{M_z`e=X$_t|6nylGvVG4sY`F-!fi5s5eWW{_Bx{s$zR zCxND?0N>Mqe>o6X0R*1`LeBzi&jI08K;(HK`U23o7Kp6_x?Tk08-SjTKw=Znw;4!o z0a7mm=~sY(R{>iyVD|xz*MV9;;A{b00l@t>P#*+5tw3W4@V*N)wE@0%z#smPN=@Ly z|0w$1Joy2b025#WOn?b60Vco%m;e)C0!)AjFaajO1egF5pm;;^hT;vy8;Uo)zd4(b zO~@u>6S4`}gls}KA)Am*$R=bHvI$G93A2bd`ogRNr`$rhh0Y0`6ShskW?$t|ypct( zQ5GF{$f9MUEE*=uqGp;b`tOs);Rj^V`k+iPM-_7_MEAoP#!beL^F~L&DHqxtq?Z#v+`#dvR8BhlXrKJPrc&t52FNu+GOBxxRH5#*(%tG_cWtkirl~)dNt4vQL7JlO zjnZA}-gIq0279APW6?-^H0lGb)U3QyYtc$N>~+{{)hBe{KB2qzgib0`pU_Lk#;Who zOD6RldP!EFD3i7oN4oLt`wvU&x<5(n(bvDpyL3}?CcAXgarE@0kx4gZ%Q&fHgKi|# zZt%?@uzUmr_5;C>fzUyq?GO+?0z{4i(c?hpCqV23&~*}se+Kk?4kW$+`c47KGeGJ~ zAbl1XI0x8Lfc+xiNCUN(0OtVUs{RuM8yir+4Di^2#u~uu0Gd_;zFNRv2Lzlza5WHe z0c~skRH+HO|HSnSOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2l2^2j6N;h0q#AQWX zR#alkig;6L-c-8iHO~@u>6S4`}glxj1H(?g(Mo@*I3PBa3 zU@8Pp#PM2%F%a`8-Asi{B8&DaSu{T;i#3a7QT=y_ILkCJaaIB?tAU2~0Og!Yh)GL? zc5s`eJkA_`bN!Kgc{p-T;!QpW&y*{E%eY~;smg2edEQ)p8{L~>-J4+t-uxA^j>ck= zt`5E#BQ3USO+x*r$8oJyVVO$f;8FeHYw%ignN?@1nP#43)(T;rfQQ!crqH@SSvtOe zd$Xfq)&FC@Q+;YYp1OC8Su@kBnI+ZD&Nw>5s+p@CovOZevhvz{%wzAL^2DMkGpZh4 zR8?g)&RH;j!NSKENH-gJQymXVL%TP=jDa@-?oIWD8wTEtlNwwZLXO^|mr#E2Sk@^X z%ZosG0}$B=L^lDQn}OICpzCEI{tD3ZDv;O?^t}cocL1q3fb^Tdz*~UrZNMG`9IZfY z2yng&xY_`BJ5V16JRbs$5y0C4G(`d50l?o01P%ki7!W!Jv~>aDZXgl|qMrhtJ#cSg zy>xFx(brY*126$5zyz286JP>NfC(@GCcp%k025#WOn?b60lGJIZ|L68y`g)6S4`}gls}KA)BzonlOudV<^l*aQZFuTd18-J7L=tZ17bc-5VnZ z-Q+yvryOq5pm=lRgNjj8apnq?LXnzECU>Nptc5ZIsn&7z+DT}*8v_U(6}1#x`3uNfX@y1*8_oiAlLwe zJV4t^K)4ZzYz3lTpmQ4#YXZ8Ofw&LodHp{tH3|QJa!CUdU;<2l2`~XBzyz286JP>N zfC(@GCcp%k025#WMNfd<4ObR%Wf4~vmDtK6-ddWsmM;3OrFjwZBIHHLNe`U#z$Ro9 zvI*IQY(h36o3Q9jn8mx%(T1Q6K^vlAHUv+^AzOu!5cBBWOy``NoM-$@7yb7Q>L>W0 z>y7Ho^+)FARl~vQa!_xw*fE12IFApeU-#Ydzu2s{&Zr)&{9)QRcVg0v0h4C#Qukx9 zH)6JoM%trMA84iKX?R8pGen2I4tuTogznoXbl0BHNx6@``J4JH4S0w0D-F_5vDa9P z(&&u6iNE=umj0*aN$uw^gbHTgw5IkBZQpdfkzwB`8DrGW1l7p-iDKlKOhel@_+l_w z-U1RkfxdTu^C~sz zW%G)@u7Dqa2`~XBzyz286JP>NfC(@GCcp%k025#WOn?b60opgTZ)o4pzM*}?JDjr# z*@SFDHX)mkO~@u>6S4`}gls}KA)BzonlOufqb@t={y!Ek*{_t<`-0;Lz#~z8N%@r-AfxU|;odN15rQF*#g7@K-b$q zJP7o(0*MgN_b!lZ15)ikIt&ba2-rRX?E9hKI6kI&Be<@E2`~XBzyz286JP>NfC(@G zCcp%k025#WOn?a#JprmWTwKJ(MO<7|VvCD-e`(%dy6E?p=0(VhkQX7xJ#gFun~+V& zCS((`3E6~f!lE}}7WJl#LIi~f3K0bpB6uQB+A55Pm`C-dQn5*7(Xtw7SP%3XWzlg5 zj!eXX$+D=KCezMQ&7`R?la>gbEOc4zo56$gt{V%JRcsllhBGTS??zTc7=6LE#mc$H z6?prBPQ1A~>83CfZ?+f5yRjXZXQ}i2S!#dtqh#K_o0ijM=839aQ?3lD8E5J>TF_Qf z^@@Y{jLf^yLqzI7J|gc1-wX~*C*U{?)W!hkF~HRYxVwS+INNfC(@GCcp%k025#WOn?b60Vco% zm;e)C0`zX^-O#(CcSG-nw>M`KvI*IQY(h36n~+V&CS((`3E6~fLN;NEHDMO-MpKwY z;B;H)woo~ta>BML*yyV~dN&mub2Fk5KM-#+w(^2_I5`Zw$y9F26*0yzs@qiM_tU+} zY35BK-J9m&+#6*Uj>ck=uFk$0BQ3USO+x*r$8oJyVVO$f;8FeHYw%ignNE(|UYSEP#;loX)y$IWW@j9oVb#o4j!soyJ6U<{ zJ?63ZPkCa|lo?fzE~=`!Zgfo|yL;2RZ^y(T-J80)jF~qEeDMyCb|f=z@Xg?{yac!# zf%>h0#|t!W1H4T@Q#0W60shy4fFB690HFZT_BId>0+CiA8Ui}s1!8SLS33|713e!C zi3rfw0VJb9>Hv`L1O^TRwxfXkINNfC(@GCcp%k025#WOn?b60Vco%ik|@88?G+m>LRW#DzVi?ytOoMEnW0m zOY<4Z8+)Xs%(>Q z>BiIlXG?SNzohoB8$EfKZfeeCmu@BeL+OU<8;NV>r{gTRsmyk7xLDZqCT z@TY;mB_KEegsT4wf{hIbF9RZWAX)=-I)KCf zfNdjS-vl@|1GQTK=gWZW6~O%}P`@4UyaqJx0K9JiO>Y9ew*ddn|Ekmk-uW-yOOy#P z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`8CqU_j(ha2>N;i~lcyn_$A)Am*$R=bH zvI*IQY(h36n~+V&CS((qSQBQEZbV^LfKzOt*h1fgz6sl=V1uvnDBVnjOd^Z+Dp@o? zCW|$TWl{Zih&anMvglX|w5$di)&spp9J&KXCgQ+kS=3CEMgM)WIQ)PtS|61E0S4z( z;=3;qn!#AOVi8mwgYNjjBjQRFAqw-7W-jtksQ>1}6zTkgZT30?I zwa1)+yt_C3HyL}135 zJ)&~BNrU3ejSnhDO~sijv^fmDxgD40R{WXDf?GH6eAJWbmxzLHl|pYeK(G;#QQd2# z3U#lQ?pF7@YkR#kP5rq{nxyUx(iC-Xla`Q6FffX62n)i&oNM zuftxeKB4>e3Ej0PbW)l6gkCx}R(*$FGO6#-OS1YznKZ0;Gde?WLMtD!bi^Ru>>Svi zck!n8RQ91a4cnFtJ@lq^ZH9OwAl_7CF3<$CRtW1)Of}Q2CR5gfdW%77ZqYzuS*b`Y zwLrKIh&X}hYM|2v#MS^^ZXmuM=&1)14M3j zj!?W2T-Ct@m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyyk(0L2@MHxzFu-cY>Z4bItw zY(h36n~+V&CS((`3E6~fLN+0rkWE-(O_)Wz(G_MDIOP_~Ep$%koUm;QHvB4&;>~mp zy2*LS&vXp=St7JoH~h!G(Y?9;$h^FII5=Gn?#-?EGdE7HnWY#tBA0Ex^0K0wG(Y0a zEJffk=8@%)+(ueed3FPxi_t; zy+gY<)s1?yRn=_9DVmKDnvGWKwwlY;V`G(L1{@QT_`Z!}P{f-A#2b7qNG!*ISQpUM z4aDO>&!<452k7esk_jMn8c6p61O0&Q9AG~WI4%ISUjxo>0M})}eFdmrS_R4GNucp5 z!22}Nv>foQ0Q}DYfoFl>b3kYn(DpnKegTNA1)}SK&KH5$2B2#r5Z?s!Y_6)*B(_u) zeMtd7025#WOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-3xxU`5%i@3C?#FiHE#?rj8 zbkT1t&5Mv1AumGCdElG}HX)mkO~@u>6S4`}ghg+{EaHupIs|nH>JSCfA$TH=*eZ;L zm`CxZ95RV44nH7^)(2&=b*3!pX3OH@99eWfEQ|IkSu{T;Q_i_6=NLuDO3k2pbNvI% zq`6+Yx$*IBYB&`_ocj82n(??o`QoE;-XVT%rE$r?$0SWq_oF-Z#-~*k8-w&$?6puxGSwU9xW;0-rr?Z`7F)F@q5jk3xYnw$Or>$~ zsDAJ@c&)k2s)Mv*o}~IK6BK`CyeYo5%F>^LdUN^U(#hsK)u+bese8wmH8ZW6SyJ8X zjH5HGnz_o+sp@MdE3duBJof%6Pb``;qw3K`RaI8woCWh2EPQ-{)OwS#H~p8^N#>ua z@0XPKPcZ8rTDb6$@7`FwD?`1}TJ%!u+}zb0d@X1!ULdgz=xYL!%|OZrq+bUH{DAE( zz`hf3yaUwk0-U=6*B-$A9#H>2;Mof_egJs)0ZktPzWspzV<2!42p$4LM}W4YK=?Qi z`2>ib06I?svCn|6&w=4aR%u75=fo}QsNfC(@G zCcp%k025#WOn?b60VYuV1gPFny`g$T^@i#V?{Cf~WD~Lp*@SFDHX)mkO~@u>6S4`} zglxhRYr-t*O<7^qfm3gx-a_ky)(P9DV6(6CsNPg6Hi;}+Rs#*|fnK95I_|)ci8wG> z7B$mk+BvG3G!ynJgovvTrouFc}fP-F7=<9y|NeK%`Hofkvt26J$9 z(%q^|qnDgx;IEu)V)zEq;#`=IO%3|=G`PyRhH`a{#$DA+PyaK(oM_hGV?^$ zttnR&pK+#MqXlgxRj)XB&#=NfC(@GCcp%k025#WOn?b60Vco%ik<+i8?Gzjx+1PCDzSA%ystFxD_!*a zO7kM*MaYYgV;(r>flbIJWD~Lp*@SFDHeu15FpG7gp$kD5f-XeCTnL_s6SfMYAm-7! zso;>C5e@jcaSTg&K`fjc#@&1;#9S}OjNao)MZuwVlaqFHm#W_sX4Xw}akLxTd;e{z zTlJ{azU$T3@~++VC$eidt^0ON98$aK_vp>v({2nH{?nWofp&xM1%>5(z_Ayo{Qz+8 z16&^g?)^ah$AIS`(0B;&9s!z;0>0yb{}Ui^0tlW2LZ1O`p9A49fXFEzdIsqH5{R7z zy3PUd^FYr9An`TO_YIJ|45Y3A>7|cCv3U}(*#UbE;BWx7D*NfC(@GCcp%k025#WOn?bcyP6S4`}geBI5S+pB{Vb*|CZK2vie2tQaK%!o03y*wiQRYajtsQ(r^OGO~cx~c~@>~&SY0^ zI*y*6G*acpB-LRUP`Nt(W~?&)#$XZ_A&IPN4)m2(ioWtZ;C=z9UkiBF0gW#L-VH$0 zM!>fT@NWhJTY%upK_W-H) zfb{#oz+S-iAz+UHjt-zU3OEk{u1>&x7^sf{o?}2`7nB=sHNfC(@GCcp%k025#WOrYoqP`TlvA}%W8qM{O8RKy!f^M=wzzo9fQLSBTt2sz(@ z^BvfPY(h36n~+V&CS(&9y$Q1@H-aJrMF@%z1rs57B97K7jDMI%c835eFvAqGp;b`tOs);Rj^V`k?#|FfpeR z-+hVD4Q{iP$3uB1`Lb~25o$NH6d}em+S`2P6hnRe(Rj8}Ib%WXBAgC_kvgeHNJe$9 zkt)=^R=Qi=>#ptf(lqtwGHH^!H%L>|y;1rHm~`Vg@u;PF`(LE?rB9v6yLNN4NjKH; zpfsd*)9lKabYrrZrT%jx&~EU(ps<_-yq^J0p98)x0RJf|0xk8h}w40lH zHWi8=V;+sILix2cZ%S_9P4jRAZ~h7$NMkWci`D%YX|Yvn66!xaj%%$7%TyW%kLm|s zgV&nNtU64JF;7y5#!OI##*8=lw*SS_dKuqWgx&<&-U7lqfyg^RbQjRM8;I=zy50lg?*l!1fy4(u z-##Gu5s=ysq(24*4g$8rfIS8{jsdk@fU_HL#R2!HKz$G3=>-}SfcG@e)Cc(b0e=z* zd<6tkKNfC(@GCcp%k025#WOn?b60Vco%m_YFppn1c! zMO<6NwM8YiwutwY=6$7$eqU)`guDoO5pv7}$2_nJ*@SFDHX)mkO~@uJdJ|?bZ*=q_ z=tIzlD3}ky6LG>;VKl@%nm5xqbdVyk^|w+q?-~{)?sKga47*_Z+L6 zV!l(Aa>hf-xp$0NGt;V>CDqN&I6A|snX4R~s=juz^4fdMWAC5x#G)xPsvccbRb@5K zSulUW!p9d#H#2WqQ+tOtZyHu*m^T9EP4x+l*{Ys&oMH_bO~Rs;I{sht1ohZYlw&6R zXEjL82@N!s%ZkQw1qd(wA80mD0@0^{&ZmLca-eGk5Pt^fc@{`K2lTB1lFtLF7l8Cy zU|=0!s|V~2fWrgSz63ZM0oPW*?FH($0iGtHu^I6CfTq_0pC9nI0D%Ayd>aS_fwop4 z90DTm0?{_0v;BW6HL>vj@P499fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)Aj6g>f| zH&kz^-cY@vdczx>vkBRRY(h36n~+V&CS((`3E6~fLN+0ru*8}$i+ZCi%sO!DE!10R zozOaA+Z1g0RUXxwa_A(oIQ)PtS|60f)|s-Xn=Olrb7ax|uq@iEWYPSXOg-nSo?{dp zE3cV1g9^^o2N+9py>@fs zdyQ0~?zPh0>Rxwkua~B&KbJ|9)V)EPqVA2-UFzN>-Kp-!;AbNS%V?xM8ufuzYF6H< zg#e<%UWdI_eM0x`6S`|p=-|;PZ`Di3#;WhoOD6RldP!EFD3g4{&Ahof>n7jkP4t8R zv2?HfU#b12_N{q0Z(46M^QQmOI?4PqRhW?!b7z8C|IosPk9?=>w64|PDA_bj{{Ar~ zcNjp~#T$GtNG$t+*hfIuejxra&~p$-90K}|0Li03>Nt@81Q<90*ggg9J%FPZs7(OQ z(}1fFaQ6fCNx<_J(3k?e7lEcU;JXC)2Y^8J|3b2{0ik6;n;i((01*ceT?us70D<+_1^aT%bzA8a%)=uu(12y@OTan&vTxc&5SRc(>K0ZjQhnIe03aNIdhoV9cLbH zVBhSUTixDnYQRte)>1G+z*3`BtXOK0TBC+iw3aBfMyMFD)Tp%vs5NT90QFqo1-lzc zXj55jy78JFKP#-HP1e=5uKnisS>Gb+5Y!>4Lrj_u!4Kk)t;w+vO%!j+#3j*H%O;?1 z8_@66Ro5-pGXpzj>8f(BPB+K!lIFlmf-9nnE|OcnxwwCk*%HozysR4&M$h2-9hVF< z@7J3%zgoGmmdS@ss&#{pyMFQJqHr@g;?3@{#G6N8>R3Exi#N$F|JyV84aA!#UvD*C zylFXA73+@NIX>~m3TLU}Pva18a9ogBwgRc=fPw8m`gtJp z0+8JS47~(+8vx%fz~2Z|H32nYAn-a6YzAsufKUXee;a6s0*$RexD9B27l^b2(GH+3 z2DE<#e_E+8HUx(@@1ZlLEVkW2u5$AMH2FwhI6lR)Mxh&S0jiZ_a@I+y?xU;<2l z2`~XBzyz286JP>NfC(@GCcp%kK*1BBcti1q;tjS<`8czld}q(atq}aIwy2a*fx_k`N~A`rj%1|#x>#RYB8mFQbe3Q z=H2{ym*wh%!k&e4T=r2Oi{w?4KXlArNbP3Kc{fdCX*Uxy?e&rk+yT_sR?zD`Se@?GO;ESPaL;3pA_-8ht>x5@_}Vk@Y~d3TUea+G~K=CZIC_#5V)oK_IaW z=&1#gbwFPTNWBOQ)C1|AK&Ao6?gEAy0q<*oZ_nZ~i+}Ipg0Cpx7hnQRfC(@GCcp%k z025#WOn?b60Vco%m;e)C0!)D34ObR%Wf4~vO|g|lystFxD_!vWO7kS-Nyw9sV;(r> zflbIJWD~Lp*@SFDHetb=Fo$hGiHiq549VY!(-F#zyKKs=BUgywnf@ZW9!gn z8(XJ#yRn^|X*avZ(rzYZ+Ksn)v8Q_VziA!suNyR7yQw^#Tf6Bxc53GMv>OM;{Is4P zk9LCtgTm4R_#;5o+dxee2($vhHlX%hAk+@jcK{7Bpz%W>+zB*y0g*TmJq)yU1MNqF zSOVxg4#aza?p`2~1bV&#l6^p5KaffR1E+xW0FW63vT0!GN5Fdq@cj(<&jD2zfSO-` zz!U#gW(ltPH?C-40!)AjFaajO1egF5U;<2l2`~XBzyz286JP>Npx_BmyP6S4`}gj1{ub7(hea@K%TZK2vic4@pIb}HZl}Zgi7Rd9_#1pej96c}fW~3K$wJBrPo)u!;xOC+j{qQH^ z)qm1)H^!sPzqxO7Ta_xg*;P6Bu4!(|eI=F!TJ`-S_RcS{ zER?--jQ7rx_ulEg>F(K&E}uQW{GsLL-c8-E5#w%L z;@z}Z$1&~(2L^}52LvmDT0an457buy4b?zn4G`W0GzWmlW*`~_+O`4hwLq*6=nMhz z7lH12Ah8qZX#kSDfWAf`)dUQLf%NM@rWwe#07DVL`wrmS2l)2`RR@5Y_kh6rK=2?? z`vDL-1k`^7@225ndN-=zYby8!m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28y&JA9 z;>sefESh2~i+ERQ-c`Ebca`Qz$diyKA%{G0$OD^@O~@u>6S4`}glxisH(?I%#z7l` zHUw>mNwXpNL7cBOITE6Y-i`A&?9q(ZAu5KKG%Vd*{iJllG@OOXmdD5&STK|ND5H0C zY0k~mns-wm?3n#{D*+5kARwwfxss~@CZ=*DG>S$s6PfY zd=5080K#7Y&0hkMuYu?{K-;%K`$^b0vD36~IQ51JFaajO1egF5U;<2l2`~XBzyz28 z6JP>NfC&^n0opfQU&QrATwgTB))(>S(!9BJ!EY|jlaMDNPeRUm;H(EWA)Am*$R=bH zvI*IQ1#iL}_KlTJ1f2*v5tHUb@PjyJYjRXX6YZN~7$v$odXKJJ@72}L`*c-(zpltEMG74V{aIj#G85w+#C?mp{Q=n#;|btDj$|i&LsBdB`_8gqwvT$4sA~ z$0GU7G;drE&(xZG6Sf*gjl(c(te&gQn~TPc+2&2#z!Fc_+e@{M4)hzR=53+%a;AA#QDI|#Y>kxyi}{M9%0@nm^T#xi~CmNhyF#ZA(tUA z)vX$_(}C!kQWj_|1ER4E0A~d4e?LhtWK*I|_;|?JF643lI5P1cNz6!MM z2HIZ(Vtatjy+Hg8p!?0GWtPNSOACHWPksR=zyz286JP>NfC(@GCcp%k025#WOn?b6 z0Vco%sNPV$p?X90hUyJ(a?U1X6S4`}gls}KA)Am*$R=bHvI*IQY{DtlggMlkqRCkY zPQ8VC3#}7cCv2NZ8-Qh^dQ&DgiLP2U0d?Deey6UwZo!@z*fC32m2-94Ifj`u2WHYr zg@|(7H^U~*BC|D|1$lWlWBWC8#FSAcK8nYh?Q>DgnIn3R-NZ++{5HoVKgyH?LW{$r zG^eq(Xr;#1s@-91^S5tqH{L%@yWN;kW6=&+j3-vDX)a!|daRlaTN}1k;|1H_U$9+z z!G*iVnF&sAkztCyMUoM;5`EPJ_Y=r0aeF00g>;4=np{KPeA(_AoerRc@BtQ0J?tx5>G6{_ge)dp9K2WEGx65)-L0^1}4A+ zm;e)C0!)AjFaajO1egF5U;<2l2`~XBzyu1O0OcDlFXHkdE-#v5%Zqq>Y2IGC;J26N zNyw9sCn2XjaM}Z#kWI)YWD~Lp*@SGuf;V9f`Nl#ef=UFHh)Gi-_(2@BH901tiSkV; zXWfiz&JXOH5s%_Yk#X{vdNV$)W<`GD%qVkhbhDkBeDs{Me@^e_Hp91Z!M72AXJWmZ zrm?)6iJ5wn^e^)a9))-FRLw7@dpCor+}=&=p*=H3^=?|X*xgrnH&!?{t$)5!u^HXF z!GXbHSr4SDfPrctT?1q`0oedBv>EVj1$@r|{_Q~3^FYlDKwt+Dd;@WN1HyZN=Dk4V4Iug^(DoM4{tgh^2XyWS;s=24_khIvK+i!S`2o;(2uOVd415fv zKLIjF;N4_DrFWwWzNUg-fC(@GCcp%k025#WOn?b60Vco%m;e)C0!)Aj(7T~`L+^&( z4ZR!Q;G9j!CS((`3E6~fLN+0rkWI)YWD~Lp*@RQ933GTi_Q_cUPPc__3zZWpCv2NZ z8-8V?cf%<+`AzsKRlN_65N<~NbJ4{3H^s`DhjW9eGQa-uD+HPqrVg>mN4UXz=-iAs z>te$H{Ct^(dDz}mp`<;yu8Huz|zG_mp!~xt2~|Cx#>D~YUY^E&8^1o{}-8c<8s+O z4y|tQh`BbSOE)+&2rLO8dmI?*0lZ%TzApj)*Fe=bK+U&6;3N<{4b*-IguVyre*hYO z0vgW%;h%x#b3o(*5d8&cd*aU!Y*qoWCxOm2KzuFGy$(n`1@t@(BsTzk8-dg_z`(OW zdJB-*3S^%HhPDIV5a4_9&t(>W{hzs>feA1HCcp%k025#WOn?b60Vco%m;e)C0!)Aj zFoA+6K` zW&CHyert)a$e?q0uYLGEc(1#t#0HbaJ=2g{ZWXEJW|#M+KYOZwf_wAPw~eN|H`khX zQ<02nqq{d@`<1h8960cb?6}+;92q>8mjVAPK-H^2&2Av@8W7wA)b0gBZvgdg0u65g zjqd>AeL(YmAaVeRz6Z3u540ZyVjlpVhk*D;K=;Q$;uE0f2$1{~==%&v9RmhF2ht~i z%ojlROJL}0z}pY_Qh@&yP&EM53<7~P+?(K!bZocz`zSYdIyks3CO+-47~z) z8v$Pv;12^;uLCvBK%fN(Mu6J4flw5vZ~gZ&OGDeg7yNdf`~pmX2`~XBzyz286JP>N zfC(@GCcp%k025#WOn?bcyy4O!E-m8HqA9kti1(J}y`>9&Z)u){JPCOca?}GyJ+KMc zgls}KA)Am*$R;d!6Xp`NtsaL~gslzR-(hR$vOvvwATR_3 zE0#mG@dBaMK)nxWs014QKzKdST(!K+5~*I!l?_aQ2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?a#JOS!A)NiQYP`{ym!+V^w3E6~fLN+0rkWI)YWD~Lp*@SFDHX)mEiZx*l z{l+>uYr(0wP;sGoLi2=eGimd$Ow@0Rp_J(A=smh>y;oN|@6%QF{kl5)fUbHU)Ky2h zu9}wURCF#XI!@KK-ZCuUT>b=;X-3#LBmQ}vGES+oXe=9n$TL9nS)gqT(7qLjJqL7d2jb5I-7f%%9YD`Z zK=Ngv?-d~RDlo7cNWTVT_5j(vz|b3jHv;(H2K-T=suied0|M^?!FHgw0|>={`VWDI zPN1<12*-iu!$71Ph#m#n56S4`} zglxisH(?I-rielWg$N1}lO{y)gE(qyay)tfT0Np#h+38>o!^gDIcbqn^)z>ZnE zs+_CS&N0lSIWUt}Dx0osmrFNSKVM|FhO;0)??x99X8NQ(R>)^@ zkrQtm+TiDwD|b3DB&aE6fx>b^6qYZ5*q1=(*FgLmp!-`OaT4e`4J5w<`o0HJKL7(i z0qHY9=4T*#4j8%scq{$_#l{QxR|8c(pr#TC_<`VhptcGKRRi@kK*J`WF#v=&1IN zfC(@GCcp#=o&dEQYB$twsNGPz;oZ&Igls}KA)Am*$R=bHvI*IQY(h36n~+U7#hNgO zc4L{GHQ-cRsJ75Jp>e{tnY77QCTcgOoN_a+2|w^|Mm&lqMZ?Ks-pxqkrdYff4C0x< z#~k_PBh{N?+2fed$D9fN{W-sUnR2)tuxym(G`1G4)Yw|{o_cee@qU+fyRn^ytNfC(@GCcp%k025#WOn?b60Vco%m;e(fd;+v@xW0(%i@3gMimfl=y`_0?>4M)| znkOMoLY{;i^}taNY(h36n~+V&CS((`2@BqYIqVxdod`M+bRs6riQoru%GTtlh$h-M zoN|-jgr8E?```%eX2d@iO^k(8tSot$H=`LgV-De2sjTCDER-LaSi|P(-w*xe>YWq4 z=by{D*)^7Pv*MC-b8%~ybCW#x7ti3q|Ij*~8GOfd=ce*>Zs(@!*r}Q0b8f62yH@dd zPQ51QPWUV5#9z4pq<#Sgp7;;=HLHNklR$P2Ftir%t_OTofWI23ssU;?0f7Jz+ziwP zfzURfz7}Yx0~$j>_(h<(9*FD&q76XXE}*>;h&2J7VIclG(A^9qT7aGikbE2Hivp=u zV4w|1zYApA|D(*3?f8#^-^-I8f(Bt_J_0tE2bms`Xy|H!wM;49C7wS%x1A9Pk|m{M|s+QJ^LP1dap29-y`t2ql5~ zuYiU=ps^nar-0^DKx6=j4gzgyp#4W6mH|4?0`V-+eI7^*0X-Ef;MsVAzSTg=2MkmK zX+Mxz4`i!=p=!Xp5%4{;qRirdb_JI;FaajO1egF5U;<2l2`~XBzyz286JP>NfC(@G zCQ$GM=-zO35my&+bL;ZWrs80_Odd0D zuE(YMHG^ad<-lBR<4p9P>z{dZJI2g7JTC3xb{e)$m&Iez4p@vQR;_6+-eL7vH5;}z zY^}x%w!gn%yYzxhyMFQJqHyEV%oT6En^$6S4`}gls}K;S_7a9O8{_a#n#;ZlT;l z=Y-A)+h)>+UzsT0%;TV&yodbE!;qhqGA>3Q_+ww~-dw(Ck*OXIJ}wXU=GXW$S6eF! z6n>8@pT}Z(n>inv|L-Z3@ME|$N^=@pi&koEt=b*N)^=%Y*TkT4c#5={#@3`Ntr+K0gslzR-(hRY&+p1K_&92J1 zcTIC!?klk@(5mkrv3GunWufeyW4w2ky!TG`O?S_JbouQ0z`Gsr zJrDR_0IGHXH7@~ymx16bK<%qQXg5&*+JBZ=8ut7rmo+c}Ccp%k025#WOn?b60Vco% zm;e)C0!)AjFaah|@C4}IaCH$^7jbpb6kA=ydrR})(gnY_G*3dFgggm3>Vcyk*o16C zHX)mkO~@u>6BfJ)bGSEH9?ZEnMO1^R2Ju3m8gwJ6K_e59+F;Tvts?box0L{T!$2T5lP)Z!Uj=u{4)^H&;KuP7|k8sXWx1ynLB_ zoSMtuQ!0Lq`&y4ud5QK7Y@4ZN--NA(W#iBuxrTis`z;s%WA5ae(Ch!{Y4Sd-b!^%G zoay#W>oq3d44&VrxqoliJDS)#x4P|rT(;~_|Il_S_KmP_+@9%LMYU}_lW%ZjFj<;` zh8CbP0)*cNnxjCZ6^OP0ZSMl@?Le#p=!^mJ4}tDZAkhW%#DU~tpsyQ99R&swK>9e4 z=>f96z)%wKehv7(0sP+rRVRU((?H-mAox8{`vVaA38+5NfC(@GCcp%k025#WOn?b60Vco%m;e)?eM9?(_6_YD+Bdw*Ih&A8 z$R=bHvI*IQY(h36n~+V&CS((`38z>S=CE%lo+yeZ6i+Cg+(_aH|C;z~nrPpYiBO`e zmQ6t2HlW|BtFBwHX9jl6(pBYLoq~=bC(VJJv{ETusmr~atDi42+r(LrmwIzeLFQWG z%y{=N7SU$<@MD2Ir*|{uyqn!)c{dk*8?(Keu=in4YZttmtbecR-c8G?BKHhKuPKIJ zQ!?Gv@AROpw2E~{?i`@1 z1=PO-H0%Q!_XFVrK=XS*Npx_D6yWz?rt}NopqA9krh&Pw!&7})|b7`K0JPCOca@GT9J+KMc zgls}KA)Am*$R;d!6Xx)4EKbG2H+C-6r*^>$!Krv5*lw5$!9?$-l#_18HRT85&4@?w zqo0K2HrFkM!ae3df3xd`-s+YEc2o1;>}))cgzt&H=#- zKP`jaaL+ysz4R3AECS((` z3E6~fLN+0rkWI)YWD~Lp*@SGuDb|EJv>Pt8zi}4YQ=4!q{xvDmId~!9LNHOg;e4C? zX8V+?-UmncHY5JIXkzr6Vr9!ixf$sKjreY+|FcXE)-}nGGWo;;4A?Q$8l^dntwk#} zw${9rn_G?dyR_Sl?KEtiE{n&a9k3WrtXk7tykhlOH5;}zY^}x%w!gn%yYzxhD>7cN zYsYUge!{M~jGwS;y75AhwyQA8O}zFIPwy!xH*Y7uGF`c;Je^y)={k05=9tP&iJ{y~ z7v;tY<;JS@&c9Nx8Qr&i=S3h{5A^K>QVqbsE+E|qWSW3%7#MmT@V)`~ z-UR$_0afn+HT!_Tejs=NsC^Fzy${qM1R6d78V>>CkAUWnfygI7^a#-QDbW5I5IY8R zeh$P>0Nq~zi7$bkuYu$@K;O4;Zc-=d+;G$l6JP>NfC(@GCcp%k025#WOn?b60Vco% zm;e(fd;)ZCxT=V&inyw1imfW*9i@3k>4M);nkOMoLY{;i@W25NY(h36n~+V&CS((` z2@BqYIh-5iR9vLbMfydwRd^vd6)%Jv=Rz>ixtRl(L{}Z?@UR__08qk7m1= zrQzt~^llc6rT?r{*6V*PmR~~eCa+O9x515ZV4})ocsB(ab(1{xh-WbJsMhgZ=Xuk; zn`@1_sYph(QN5d{fc;9x#_F+at!ph;?i^jb!I42?830m)z(5*E{|ICE8qPjSMj_}8RLXXS-}3&BM3#`zl-X~ruN6+=uKwr;L| zQaWKC&O&9$W8lp-U7AucWb701D3xDJ_a-m*=Av^mMckW5AOcw+0^MR#{mo9sF zsn&lo_spBRU8_f*c~f0I!o9J399muHxZE2Y89bJ^fov2QY6ZLp0N;Cn|9zn9AW-uG z5I6(`KLTn$211_z^+$k)Pl3kIfbcP(`Eww00*HPAw0#M*e+|UG0Xn}0;wORb(?H@o zpyzua`2*1R6OcLs4EzkF&jFbWK=v14=!w7L`>h6izQ2}P{FQ$#_^JYa0Vco%m;e)C z0!)AjFaajO1egF5U;<2l2`~XBzy#>taCH$^7jbpb6kA=ydrR})(gnY_G*3dFgggm3 z>Vcyk*o16CHX)mkO~@u>6BfJ)bGSD)r{ZFLF4nixTH%G@RJ;&woD0E3_hudk-Q+#w zXC8+9tW=hZ4-C?}TEDq`&mvP*9DH0J`b}Pf&0-N|9Hu-L$nT+hGj-jY$Kcjjp_&w7 zYs2<;hHK-zZ`+?x|Sc1@al)0#Orx_eW%afEv#(}yZHTE{c@21f>u zWgXyu3aEM-sM!DnHUhzCfZAt)&=#P6E70&9(6}84KMyp&07P~G(U*X>mx1ZeL#9YkU0Qk-vfr;2fQ)B_aWf#1gg4#nm7tL^DdI2HezwCOCo5O5)w z=-w2=C(+f>dvw)$uda69r>pAwb#?XuUG+YwtB!JAH7(KU=Unu2oT_X6CG}?5!MXSZ zlW8vZZmxcQohA-O*W{tz{2HOAtPuLl^$0sZ#lDnAXv{;a$_! zo7QWLy%{{eRdfH|7@MQX*qmG4_CGFL_NRZ$%W2&*LcMW$+*8wWVJ-v{)tfT0Np#h+38>o!^gDIcbqn^)z>ZnEs+_CS&N0lSIWUt}Dyy!nBfOgt z|6F9YhO;0q@5Y4bGfuHu-1$G|$(v31+;RO9sy9V$C&U`NcKjy8tg&k@Y;D+~8!r@T zyT?**M9HyW_{$1oJ55_rVs$CwKRfnYON2!Rox^+W!|%a+-9;rfh%WA#hVybOo|fF~ z3hj8z)AR$>o98baGhMxDIaTDIVSHM#eA;wZzte-l(kj*+xpRE#joUL_Yu#oW&(s?n z88nucfQFZW##eywt3dN^Ao3az-2=4k1=`;LVs8SSZvpXlfbM-jVn5Jx07$+E^t}(H z4gv!o0O>ZkOCS{0pS6l zc@XMNBu({(Q*W366JP>NfC(@GCcp%k025#WOn?b60Vco%m_XqZpn5~~hUyK~8>%6S4`}gls}KA)Am*$R=bHvI(bH6XsBFxY(Z73H2*Q>%^&eA>2?Gf{E%) zDJR{GYswGYn-P!VNl|g~n0Ye}XC@!JW}$dB(?@wMl2<9-U>45Qnt9VSmUwefxG`J2 z3IFhzr*;2-X&v5ge=uFV8BFCCZ(0xSnK7z(Q?YNv%o_#qrg!aC;*DFYcyGiY90$e* zHKi<&Sbh+RNfC(@GCcp#=o&d!g zE-m8HA}%eOVoQs7YiZtEy5P5#=1IttkS8H0J#f+kn~+V&CS((`3E6~f!h$zp4)MnB zR5#Re^BZPl1yvBLAiOr6icXaE*IfuEiZ>i`liz@!Qq}w52;*kNKNn4mfK#kg9@5Qd zg3KuHOr9^z5yeK0c;Jgow0AQn*9@Lw5rQ0({3ufnjE8Y}l;$+H7Om9STD3cj?bK3l zc8#Up{C(!l#jRO0Z`$_%m#6FL$F+_x53l?0?%Rw3H8*2G&0W*nmitO93$*I{N9>(n zVp%AA=NRvuCGWk{ebe2uA6-6se)&Vo%galg4=i20blJm8waU}E)tjzkr)G|+-jo=> zf4U6Iak`Yps#T9LZ`>ZY*5w6S4`}gls}KA)Am* zIK`SUhk3(=?Z0wiJDmWhVssSOY`hSr=7nIQc{2w_iLN@zb=9;)SDRPps^W3jIIAtX z>RJ!9Yy#@G0sT(wx&?b?V8<+7RnFDb;2(5#^d4Qc-mCuxM(32_*e};l7KnQ?astlf zd(4t?^t|kwk!sB4U)9{_D~toz<=9Q0voq!|Ec@s5Zf-OD9G7;xv7Lsk(`E5kv;!97 ziB)Twi&w00LTuRDu(cX5*#7>4?a~W2t;l%6t{uP0_zAn_GJe9Y>Bb91T7gF1#GiiL z)B8ERn|E7ZFx|Vk*2tTRWK|)!GXbH*#IOq z0zJj;c0NP_f>_ecl6Nq;K-Ekms80hH+l1G8Q1iYKnae6n3 zi#nJ96JP>NfC(@GCcp%k025#WOn?b60Vco%m_Wf3pm)QSMO<0Ll|@r*Wf5;I%^OP> z{KnEe33(FoB;=e2&Us)HvI*IQY(h36n~+Ud@FvXR-8eWR#;NEISf-{s@atxM&;;RM zleK1|cjNpGdo<&9h>GDQjUup=PMC(XP}%aBdNY~sQMA8$pC?~Krjo`o(DohKz+sEAlZ0<#??UB2Q*g# z5kC-J542SQ?bSf62I$-b!~;P0W*`v+dbR<{TA;5ENQHob7lCv=klFdSGE27MZv|gc zz%RfAm;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz28#T$w@6mKZrP`u&I&Dn%(LN+0r zkWI)YWD~Lp*@SFDHX)mkO*qAxFo$?U=Y-A)<*OpfS5CzX;fA^pOcZbCamY>H1AgXV zz|Tr$yZAt@j3VLSrA4NCIQY2y-5bbxE0uk69`odbIQGWn@YpVCICf2j*El>y+Dv2X z&}JK3=cREslRWn3F?_hy;euu5ajP5#z(5K}p8_%iKz0xq zN(0`X0N)wtH~ycg-*D~?6JP>NfC(@GCcp%k025#WOn?b60Vco%m;e(fd;-*OxWI@D zjJUvPiY+kW?WK8p>4M*0nkOMoLY{=2_P}WmY(h36n~+V&CS((`2@BqYIrJMVM~gTW z?F&Wwf)@fWgd1sMl8O3FF_aQr9lb|at@r9`=Y6`WzF${oAJA3rgSzS{*HzOJor=yy zMaQYS)?0=JoXekJGR=g_IHk&xhkcX7xLGK6jAMcxi{v*`y_s_AP1tHEH4a0u$)(<0 zRBp^xZ@k&Rd8#{BY8~IKI%&Fk(|V1`H-qQ5YVO|~29O2==&oDc_CGFL_NNl(14|b# zUH0%&t-5-IdgJi8wASs`@l3wKkwIfQ5BP_Gs*07+Y`j2VH4yXxwUt1~57e&*8mfTC zY9L$#G;abT0U){=XbS@E+kjXt&{+q>LqPY7K%yS#*$E^YfWBQosu37y0@7h1^E!}i z28LPy?^}TH9l*a2sM-(I8~_6Ett_(y-(Sg_iZTHvzyz286JP>NfC(@GCcp%k025#W zOn?b60VYuJ1gPFny`g$T^@i#V?{dy2WD~Lp*@SFDHX)mkO~@u>6S4`}glxho)`U6K z8(Jr{PN-kmX`S3i)(QWb_-mS|-js<=qN|ooK;1T=->IvvTd-#acFfXMYA)?&&&9I5H$ZQQ~L0;a?*nZ6%F=bpPK8oeHQN5XR>do%4)Em>L-rQ+8FSp6m zn_FBp9Vey0bGrB$pua_9Km8<)qe)qOQC_XdXs zk0k~KKLlz!flwDv9|sx^1C8B4_$bhv03yeMXb;fV3$!PJ*jGSj9}w>cx>G>n6wosO zBnN@MG?4lc7{~zWvp^;bWX}UbLxA^*3JXM=RYL!h!m2gGnzh2fI$`iBVeQkx&<0`s z#tKWBrQw+hOTiZy@Cz^jCcp%k025#WOn?b60Vco%m;e)C0!)AjFaah&4TlSjxX_3T zji%T_Bi?A5H<~W^jiz}L@+9O*$hi-k`@kk-6S4`}gls}KA)Bz^O_)Q&u{aei3|bgA zS{S?#oQfC1jdLNGsNs}y>dm+&{=mZ-@hFx(n16H4hbdZ49`kR;C)ljGZiPmlbOQ&@ zPnmq^*v65$?rcv2|#(jjdC=-PpP= zZKq-DbXhzWt=8HLRH1#ftH`LJ)8~O<#$H^YXEv=@!mlDz}BxdT7s#Q7xR-Eh8r2 zxINRgikEHUnSdk529ss8upuaH+$Idy3Y+VMk&rO@qOh%A*uGO3YY=ws62=>a-A%$o zSlIKrFxf2ZYZ0a*!hyGi>8LQ%D$KSChu#%>-xvB03jH4ls}2ckJ`x5#76v~N)*cat zJ{8t~CTut+Z2VjpJ|S%WLJXbAmo#)NipxKk025#WOn?b60Vco%m;e)C0!)AjFaajO z1eid<6QH3(Lx+YA4ILUfyeT@HkWI)YWD~Lp*@SFDHX)mkO~@u>6S4`XSQF+jbhsw} z##xh3!@{Zf*Q9gj;Dvw-!9+uclX3E!3RJ3k9~`0PjQHmwH6f->v9ji2=Umsc8bh_2 zzEX|ju|nQ_T_5@@ete49IsXbH#sP=sdhMLl1lT#d3S;MlA?A2m!%x7@@xLE7-Oi~z zo!idoI(BO2`0N}7JEu2#)r_3!TG#dw%8tY1(3(aQwj4*Ll*pn3|&kOs8gsF-rzyYr?y;_*@3A2^LA-~Z3l+gFI(7!=gwNY5} zj4<%5Ft|lnyHyx^PFTNP*zmlt@daUchp_o2VdQ0D^c7*-tHSo(Pr%NJz4ip>=g`h! z0!)AjFaajO1egF5U;<2l2`~XBzyz286JP>`Pk?p~*B^2H5!WA0vGqs1O*L;*UGUpf z^CaX+$diy$BRDmJO~@u>6S4`}gls}KVZoa)hn=IGip%-AoWF=(3oitx;)QVITnHxG zIdfp0=&GY!S4~TFwRwfEDjtWiv)ZDouJu66CZKK`(C@^qTd-#acFfXM72{y|qq z@6lE3z4~up4o?}5eWkJvKNiUIk>hkO-(!}rqvvJlj6<#&)5Iwj`$moVm?Q5oQ_?Ax z-!}Gv%#rsLeu~ai_H@RaqVotOBa6qSUEEI7R+Ly>%J|QY{nip;kwNG1Ui8kL;(>WQAneORaYl==qGOCU4>4fd>5@X!YbQ$;KbSd~%UAJ60 zL}$9VJBP2O>Byl$XL(&1YZi942;&i9_uIlmRM^uhOtuO8-W8_Wg##VJbWE7}P?+r$ z4s{8=p9pcwVQPXk$hP9jVDo4f8n1&sltDlrkn6sJkj|Kb&WcNd&c&@+ zf=(Q6j;FU%1f36Slcozg{U>t|(y80Edh|g$75hdAI#!QeYx;l3Bk0JXL1gg?6RU+i zK4G#_*yk6f)(Z!!gz0Kwrbd|EBpeC|y<3F7twR5E!m91Un&*Xq7lgqb!rGUFp_hgA zuLv7n6*le`hF=pl?-54!3Zrib+ujtmza@;lBkbHKjPDnA9}p(q6ZX6>Odb^Wec*+l zlRD((9G$}7BelY!3j05P0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3_zCqO}mOOUt( ziA#{C*b*e(u9~;2F8J-Lc@pv@)bg{4$Xe?;gl!TbX*RPO}oR;Z|vG!V_T%nG`0?Hwy||;1<`aK!-rcv4y_1V8@9j0 z*5YwqnxEs+%++*~FmybF+g3r-dDWktG)<>9b8vJ`r)#hMN+rh*O$YPAuG~5LARRe2 z_$*z*R9rZ4SeWh>W{wK83E|Ljq4$K)_l3~^rLgL2Va+$fz_-HSNn!12Vdy(y{rAF# zAB2rR3BzZE%|8nx=Y-J^MR#{mo9sFsn&XpQ96U?w`%U+8$Ykf&)@2{|8d!} zKmFqe2SZmmIu5Pwn{hcha%^x}YK7T4;ZR8E-68b7B=o;5ta?RQ^QthgTNr#zSi46U z+AFMoL)h@9u<3BYIx zUG+P4)pZN@%)pLWx~iP3Q{yodra4fURw|pWY?s?PS3h55wy(1wKS4(qW5zLQj}`J+ z9HNug5S_^sbaod;(DA~}@l?OL8iLMuU%YI(pwn`y$UVdGaEjsKluUQ^J3Z(ttzzAg zJI5#JI6N+`VwZJ1Lv-ZWAhMhm`o9xaeJ`x}K^XW+7(64a{aF|~C#=69Z1_dk_{3^3 zyh_;oq%g8Z7+ouDTPJLPN*H@u*ttO%-ze;UMwob3*t12L+$!vQPMF#*9C%)senFVo zAzp(y*u;D#nVn^_nkOMoLY{=29l_ZVY(h36n~+V&CS((` z2@BqYIV2ssQ*l*4SM^(|x9~!6DqaXT&V^v2q{Fc}`Hc)JRlN_6uyaQIbJ4`eI>pM8 zhoCc>i?c{f8n?NRh4Kq&=S(>}2h(93@M|2JSU6T}i?Fp}`#WR5^OBWgww)7#nB!?W zB6iMeZ+>RFol|)_x1G~NfC(@GCcp%k025#WOn?b60Vco%3ZDS&9NIawb7<$#&f)FS*@SFDHX)mkO~@u> z6S4`}gls}KA)Am*IK`SUhn>Uq_&3gaeA*RG#lI$fI~y+qTnHxGIdfp0=&GY!S4~TF zwRwfEDjtWiv)ZDouJu66CZKK`(C@^qTd-#acFfXM72{y|qq@6lE3z4~up6i*qB zeWkJtKNiUIkrQ<;-(!}rqvvJlj6=YgXOdjwoODA6PEeVA=*W3N6TPQQIpDWAJW6vK zTZ{Iu#@4FcVQg)ewsvi<@$VvSrm=NsvyH7&yWQBjE^Vh_>x4&R(NY%UiB)TopR{_c znhjeUwpQZ>+uvWXU3$R=Bb91T7gFBgyH9ST3_%%(y7_; zz3Gz9wMOVvB%|7>l1@`##0VV)NvHP*%axK&&YcdAOB;OJ3Yi7R2ASnkVdOJm^q8>i zb7A`lVeAWG=a<6x*TU{^go$s3Jtu|9)55;*gsJa^13w7UKM6BugxQ~kL+6CvA)&9r z2l~Cjs@1|8pD<7<4Elw&>xH2zVSTl*p+?xaNf-_Yn>Pz1L1A>8u&q|uUgv|P6ASq` zLZ|QxFbaNuR(=5{zyz286JP>NfC(@GCcp%k025#WOn?b6fr2MMNr%gjxD1KQkfzu& zB;Ki-cd9PbCfg?bAw z1gGMKaN}GECQ3Ta->`HuUg4-164S7ubM=$b33GN9Dr+8NbgpNjjasS3{V0*moomdW52aTSY*&Syw^Vb9=z9G zRAO^g&UKf%txA>Lj;AFaR~zIUPuKTi==@N<_J6u>GhVzIFWxoHZMm<+vOuf8f5hJT zC6&PdyG!quGOQD(P`ac zcb6Eyf4Y28r%S~mnB($tmCjtdF3wm{F31B1r0Ll}EW*!i+B{)({s zRbgVcu;(>la*wcYuQ2t7aNtd0`YmDR9btB#aA?2K`>xQ}F7$T@t75{M4~2nFVX#YB z8yAKS3+uau4M&BI31RrSu(?MV=@mwk!nUu3?R~;nzpyhUjGq#A4~U+V7^I%V89GdW z2`~XBzyz286JP>NfC(@GCcp%k025#WOkfHmKs|?g4)q-BIn;A_zjQVsn~+V&CS((` z3E6~fLN+0rkWI)YWD`!YCd{Gda6$f!vml>(g;VjbN#U-D7XmH>6ZM>VoST#P%%FLg z8MIPaFFqhWUTx@HzGsoCwhlfn4?}0%icL-{XMuP(mkx9ECyV86=6q=WzsJyma!52e z%^I_IUcaGpn_>94v|EksB+bxy3?F9oI52fb#_~9{-(hR2xoX!eH++1$wT>bnyeVtOJ z@(^_LvTAaE|50|9%Awh>^(d8>=;&a&j!nD6aBu9|Tw`0L%`~YLn^cgsu2c z3!I!su+8P@$bR#u=p>=%cm|)Ugrl>!zr}P%r}Y|BbOz6F)!e@~CgNx^h38hc{g2C* z{pla$cPjSTuN;qKM>JhKEcr`1a%7NM_6t)7gahvh)9(v22Zh-WghPjf-Y%gpF7zK3 zR&@(&jtT<_Veq)HwnrH171k$(4POZx`-I_sVRK3tIVFq^2-^mQ?P+1`M`3417(XlQ z&I%Lfg*`*UWJM*|=M|<_3kQ6{bfqxkuY{zNU0+E_r|=6fDlDq7|Kk^60!)AjFaajO z1egF5U;<2l2`~XBzyz286DW8BlyoTRP|~5KLrI6XOlK3a3E6~fLN+0rkWI)YWD~Lp z*@SFDHsKU&!W@zgm*w9$%kn8%I2Hez)b8xO5O5)wDCv}mccQD7O+ei?px>#hu3NBY z26oKSRpnfr9*^NL&4I%NAw?f+$jls|5Z4=hi3Ilb*U`SZ|qA*l1 ztlueYXb?8;5{4Uv%}v5cSQvd>*w!p;ZxO~K!p^sa@u;x7RhVcK_Pi@hwhQ|@gsGTt z;6q`$Q<&)zX5+%4!$R+;Lf>bi=lG9N&#_R?VFFBm2`~XBzyz286JP>NfC(@GCcp%k z025#Wg-?Kb4i_MC0TLG=O|bto)N@KXH)mWkgRVAGiYG z&`ju)GWt1}#^#JYK4&uhoTkF~IbNtap6Z6R@N>@W>o?uc8BFE&b6OAWnK7!LGZ-2% zKF8&8X{{~R{Kx0Wk-=l>5&Cy~3gfGV-9BNWQrP1cCf5u5s)VU(;XsWry-An}2(z1oLqVbU zIiYX6(Eq%!>IGrV4q@P>weWL-FR$hJoWifYDEPfu`30B&6JP>NfC(@GCcp%k025#W zOn?b60Vco%3Z4M{9Qrx*bLi*L&*81o*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IK`SU zho8e0`8UpreEJnm#lI$vI|nZWTnHxmIh>c1->jfg)%)PpZqDU<7EO$&Q><)xxH+zs zYEmC%@^PG?(|#Qv)r>Q+HO}oTIN){9o?dj6pXyW6;fA)7+N(N-PVs>ib9R zonK;ED0}A^@0}&@z0-Zu-LoHEK6`%oL(9v{OPmiZUA%PJ!%MZw)4AQ8u4AWWj@Qj` zYjqdK;pWJZ!C~1Y3^odDn}ngTu>N&nL$k24MHr3NfC(@GCcp%k025#WOn?b60Vco%ra%I8 zbGZ75tB<()Xo{^q;w`Fqi|T^kqM9cmPePuAoEX805o|&>A)Am*$R=bHvIz^`ggM+C z<+^HGqN~j-bXD;<%Pn=W@2_FX&$?brLqco?nwP^opY^|J~qg2W5&}vFNt`O85 zPgA=ee$JcEZZ_S|xz_BQieyw9)z4`PjF_F{_Dt6*+HCpH&XFU7$C4E`oEJ6@3BwhB zu-Pk&tQJOn!nR6byI&YvFYK%m#;b+hHNwOuVNXDq+$`)13RByJ1GU0*oiGy;W?vKz z)eF5Z3w^H${jUnEb_;7>69)DOgL{RwZwNzg3hUnzHoPNj+$Rk07d9X8!_SGl=jZGk z&dy;1On?b60Vco%m;e)C0!)AjFaajO1egF5U;>3tfPN1B9Qrx*bLi*rKIv>iHX)mk zO~@u>6S4`}gls}KA)Am*$R?a(O_;;a;fnklXGK2!3a8>6a5_LZ`ijP zuWM8ce`#3Gx%x@zgy}j9l_ih)IoA|&N=2lZJ`s;n`L$GZrrZdfrm;roJOaPS;&Ev! zjO{dSMTym=jQ{M|Z!HlP8FUWswGY1s?{ycI*x=>3OO44lmlT~a>>N+)rgczsf~%TL zS9JPM<{qI_w`=w2BXp{(M<_Z9icarNi@U`5_R~c$a=Mhqs#VlC(k31e|#=V4*ITiAV6m`DhFjti4L!oFT%Dk&WJN|^2w zX8MKMlyK;j(EFXx_r1{ngRtr+Va*v~;Adg*oUrzSF!YPC{)u&9!zy9plfv*CVe?vH zWSubjl(6k-Vf%)4aCBlD*U`}_{Q8Ruiz@8@_yw2%6JP>NfC(@GCcp%k025#WOn?b6 z0Vco%3Z4KR9j-#+DkQE#nqsSvc&lpOs=DB}s^&?^laMDNCr5B{1e=ge$R=bHvI*IQ zY{G&!VGc*f=2TqN&qe)q+AX{goQfC1jdLNG=;+Mj+?>2;2F=6Fpq0vU@qxKqS8F<# z?^$H3vV)JyL(|Di#93^fShFB+L1(eN(`w3Nf&89--Z4728RK`RX}7zb5MeA@%3?gR zYE5(TiWQ=W4O<(wR^tWR-(Rp@dcmd@886ti<2M;UVb^5DkIiG(bmN60ZP?Ly%y@$X zKOt{$XureO;&EPbbIf*h+Mwrnx{~7PoEvz5(j1-E%)!weow|+ot96{N|8eC$Nxu62 zYbZK$WKdZG!q{eEXHXd5ChV>iChCMeAz|`GVPCy4wNp6IAWZKPW*UXrCgD(6=-n&y zy&?3!DXe-+So4lBuumA=FRVQv4813;e_z;eP}ul^FnmbZ{E;y7u`v3Hu`Pk@RJ z6&)%%RCK85@RsRpLN+0rkWI)YWD~Lp*@SFDHX)mkO~@vkVojJs(cz-}8)s2I6$_{0 zUz5_Eg%<)Y1QQjVVrVD2I(m<;TJP1>&iiy#eZQ{GKA@}K2X)m^uB)acIyIh)8jn+T zt-oaG468a9pJ33<<$BK5&#zP1DOJ`y44q#y(5B2hcg7R}XUtzx_RnePTr_iB+GH3y z;jyOZT%4X`wxJV;oa5rsxcw->SKPZw$ZDF#P7OTiy0QE?f4e z66XU;7cX7*@KUXH%Lqfq3Y)2E$v6xhIWm|mUkMX^!k&I%G9~OgB}@$n2L^@dv@r9d zFq;t$ofUe|34IrY{$GSuPoPmOHLHYyCxyW^!rHaM&^lrLQ^JO)g^e48;f=!PXM~Yw zh0!g-wynbU=Y+BC!p`S~@fU>MJA{dsggq|{ldlN-UR@7EC$)P$hv*c3^+mz&(8@2s z1egF5U;<2l2`~XBzyz286JP>NfC(@GCQ$GMXy|Ya64xMc4bl`_gT#AP^Ip{jzgIO+ zLY{;?2{}4~qa)aaY(h36n~+V&CS(&9ya{s{Iz>*!CH-8|@1Wbl3js4Y=(gN27lMg~ zPMJt2x@y@3)NKR$ox19}1$$;-$1Gh{&ebXL7y{EA2u$!tbTLG7dpZ~QFEZQK!PJ;M z^c)i=)Sx0(DtrEqdGclxK6hNdgoe(TGjw*3W#~L&%+fL2&`Cnh@eB@$q4Q4r8Pg4& zmQzLU8HR{c3=yYfx~t#mL1k$b>yF$xrlE7I@q_;&Kj?DVF;1vDY#E25BZmf+B`i$6 zE*xkUrdx!Wh%o!Ma40JD?iczF2>tH~tKJvZ925pV5C#tkYd;c(J{H!0B5XJ!Z2VLh z{!G|>Oc?oG7(F3u`$E|Mr7-rju=5*X{99r7Nnzr&u;)8r@_S+555m+>!htiw^v}Y~ zIZ<@77pUlPhz=8A0!)AjFaajO1egF5U;<2l2`~XBzyz286PN-CP|=~HLq&&*4iz2V zFr7`vCS((`3E6~fLN+0rkWI)YWD~Lp*@RQ933DhqT$E2cg_@S4o#Iry5N@ao!9+!; zl!J4|H8kjIo1}PBl$|^ropHD|`B*p$#lLZk@>nFVQqXZZJT~nPI4=&2W|?bji?o@> z)}hTdwodJKV>>w`behHzbS7qmP8x2Gr{d645Oiw8Pya9XZH5$cvy8Qw=C<5dVp*V7 z-#=pS{1VGT**nL0?<{%mo$j0Np8e?Z+4IXET3%lMk4ZR#soa82>!Cd}Miq1__SxN6 z2s#drOB>9LN6?W&gUC|x6qxl2hgJ){YlObFLjO8p)l?K8sAv%>l< z!iKHF#^;3L?ZW2gg^?G8(H+9JmxS#v3uCVcJ6{#XcMH2;6DIZud-e*GZwUL|6sF!1 z4!k2w?-OSB3$q7=L+=T_9YSC1DF{0L51--;9nR2U0!)AjFaajO1egF5U;<2l2`~XB zzyz286JP>`Pk@3BmmqNo5|6S4`} zgls}KVZoa)hoED3svBxq`VBJzgUSe%5nh{4MR&^j>n;Qn1sx8}$!}m#sp@@jbRA{U z#E3e@O64Kvj3&~I;@IT*(i~B7Tq7REvWI@ol=E|TjpgS|%m5uP)ErOs)++cpzdZMn z>3&Y->D+!!*RfMG$LHs`JZ`P)E$cW2=*XeLWBEwv|5#Y{iLmB~Fz~4`_?fWwm@xFY zu>ORw;R|8om%{MZ!sc&;k#B|3lft&s!uIckvG0YQKM3PL3A@h-6F&=k&IywjgnhpV zQ%_WZ1FMATCxw|c!t7e%&^n>_|7Y+0W1Fh~_}_PbY^M}H+EtOG`}l#F5dOJW?)BcU z$#N5un`8X^g~|QH@Rxs(d=`_h5pGDhYCbW{iggw&vp|IcWfrVhp^TzqRxP#agd%fR zm{@Smiena-vueeP^S+-4oG{APLAUm^&oh+ADW@&ndG4I&U+>3xxj|^`OTK zdN;bIpX2kn>E~2_?M0J8RQ3;l04Bf$m;e)C0!)AjFaajO1egF5U;<2l2{3_*CqO@k zeh&Q{`Z@G-c(ZgiA)Am*$R=bHvI*IQY(h36n~+V&CS(&`O-Vl#KA^Qv^}PYjVm@d- zBX`WzMDqeo4Bf4Xr?9{WlmMjngsd@MtyT*{7ClA|+Ppc&=Ul<}V{ zpypULVIBMMC*#!y1U_{~dhxp&ghd~KBUDA-8q_=Wt?|;}91hU2+bkaG=Ga=lgP_y* z{DHCyI@cPY)0By+V+uO0yG9Mru|m*snkFFV;LxPVvK91g1N|?9f$d=M6)^NF7=8_m zybiYS2BUkx*j_NP5A1vsOzsD}-U3r^gFWwp=>uTzdtl~$u>V6a`w=*J2+SP@^GCqK zC*bg>pyMd$>;YZJL3bMToCLkSpsx?~XTZQIFxU@4Cp17o$3Q`c2`~XBzyz286JP>N zfC(@GCcp%k025#WOn?bgJ^>0kT!O?UNL+$cWlNBFpK9Kxy5je#=1IttkS8I>MsREd zn~+V&CS((`3E6~f!iqOx5kaSx6K51b$-q#ZlEL+j4xuu_-zICVjDn8xUo6~AR5q%U z0@H|{bM=$@N%M6U3sVV0bfhjbrH=**Iuvw-u+O$MFOY&xK)N}$w*70Rp!43(J!Kbk z2F?_ppwqf*)z}ksh5}a!IVEY?urJxgSU&{$Pm0x^O@!PZV2VeqBfC(@G zCcp%k025#WOn?b60Vco%m;e)~cmfo3DCkhop`b%Shj&Y76S4`}gls}KA)Am*$R=bH zvI*IQY(h3+l{H}zL5FS%-4se%HI%dz!3$wpT?l0qbQW@IPRSF47Rrf1D~0LeL)OaG zj?U$KmXy`kkzZH3qoY8bnKdNfC(@GCcp%k z025#WOn?b60Vco%DxUx~9WF%TLL@Fks)JgS36UXn>MEpKY?0i5I zEe~qq{6m`Pdsq_(8#U4Pm`0W7qROKPxrl#6)VchLoOE-!qI3208L%RnfNw^-g_jEP0a7t@`^d>TG>GLvG>O z7}Tsme_~YI7RXnOHlu2iTa(-x^%qQkzF@laf=LQD<0v|jKNTIQI7X@Jj8=3msyXFW zbQ03ev2}k2Mdz6#&i}=FtFFM@Bo&yuW>^gm*bR%+mIp`ez29zFjJ@;q_s+$8@379i zXWo;`=iT4<$nwTUyYkS|N0%;pe5o40#w4Ag3!7EzKlIP5`1~za^FNm@``2F|Y-zbl z(J`tm|Kmy<$1D}6)-BgkbZ}@=WqA%vJ`Z-S22(G9J!`@Ai(v0MFtY*de+kTP0tYvP zxvgM+8(4T59NrE(f}nFJ=n8@EU7#lndfPx>1oXcF2HL@32N;Ti;dj7D3~Y~s(F7R# z08Dg(od>~W672dIOm%@h-B5JWDJnXK$}hjD_-$JG126$5zyz286JP>NfC(@GCcp%k z025#WOn?bgJOL^?RCK85P|=~H!@H)l3E6~fLN+0rkWI)YWD~Lp*@SFDHX)m^%9^l< zqC-1{b_z8uGwqb=WT)`ADSw+~RCF3(ooJ$Cy@Xt$u89Ff6Wuqj;E&~X-Y4uY;B(47N4KY`vn z=sOSk3t->^7#s#eO)pA@9bjY?*zN?Q&0x$0Cf0zRZZO#bc6q?mda%a}rZ>7J$diyKA!kQ$b_AP{O~@u>6S4`}glxi!H(?P)$Djxr8ZtJCwIJ^gRL_p^opsO8pcYvNK=zRzD#Xx@?3?#td z2VkfZ3?Bp|NwEE6Fxmyiy1_&W>^ur4d%&*aU@8svoCMRoU~eCo$$WkN^!G8agy|Xz0+; z;SJN-gls}KA)Am*$R=bHvI*IQY(h36n~+UdWldPb(BYc=>9Z!EhJ_;d+oW?>%L_qX z2xT;MI5nrVi9z+E@8MBu&ZvJbnH*C`4yq}^&M}W6wT%0cuG-YD6cc!?z?*OAORwYO z|DkryY-_Eh)@D&JZfD4?VlmhZYSy4XF{*6~yNnpsLIwK@ix`GO?Aa@Q_Yi*Cm~NlPL1Hy2sR;`kWI)YWD~Lp*@P8u!XkE# zPz0Crb2)zvy%t^wir|GXeJ+GD+Bx&3b)t!bjhbkCOcNVdXrk#UY3!^rXrgj?$T-uCN;|LjP-)`;B(6 zMyE2e*F5qb`Ce;{-DGKAV6C?rg|N?-Pwh5~L;5+kmNo07r1RW`Z_6&}Tx*0*QzoX4 zE$Ku?jnJ{!tZK`_2}wFQHYu~jL01BFe*k(qLGMA(mjwMEgMltE*bRnKVE8B)=>gl1 zgV8h?I|(Lw!OlJ~nE|^_fvJA5X8=rR!QQiAW)SQj0<$@A@Fy^r2lMB_LIE7U06P8v zolmcmbUg#Qp9MY7f!^oWNlC}IdL1PlN;*t{2`~XBzyz286JP>NfC(@GCcp%k025#W zOrY`!P|~5KLrI5{4kaDlE}c!tCS((`3E6~fLN+0rkWI)YWD~Lp*@RWrgheDBF3X=j z%kn8%D1yIDYIjCn2=YQGqokuq@kClUb2ZVtKodiEYvTC5nuy=0iJcE+V2u$q)d%|G44eX78nK!`x zb}-uk4o1P;J77Kr7UJM=0(5)`IzIwkhd}pX&~pUzeggVF1^u6afn#9sa~L|IFKFl( zD!=-o;`eCf55NSN025#WOn?b60Vco%m;e)C0!)AjFaah|@dRk-a19dIAaM;+m90VI z-Ku%F>Wbg3nkOMoLY{;i9>L)eY(h36n~+V&CS((`2`k=&MGPI2BDkcVOZv@pTX-QT zf)~Q{xe&@|=q%*uoRY@|EtCe+N+B*jWT%WWbVmJiNm+#*`E?~oIz>F1tIe7l-?I>I z&a8_M1yAVdRIjJ=Z#gl?XscCg&$!RJ-K||{f&<-Z)~(H4=sIk z>9WU{s@L*#;`u{kdpg~3j`DP7Nl&Nm=!85S9Gmo6PJ*FcFx&@5GGO~DFxn5s2EarX z>^uu52f?l(FqH#)ege~Zu=hNeDS-VK!0a$M*tA|U=K%Aoz=9JTZU!BzLFWsgYc1%0 z5%jDBy&FK^OQ3%f7}yL3w}PQOn?b60Vco%m;e)C z0!)AjFaajO1egF5U;<2_@(Iw>p{GMnhn@~S9o{vaO~@u>6S4`}gls}KA)Am*$R=bH zvI*IQRn~+>JRPpgpFS(|=~*a(zfGEV23`pALMWrBQ^)B!rB4o0ME4rQh^ce=6FKW< zl%_N4pEqdj)C*Gyn$9J4X2hYH>?aL$bky6V_EKvzUD}$}1^T~h)Z6uKtvXNND(Y-~ zYq_+YA-9UfU^A#$gZ{**w&9aTn^85%tx0Z;`U|E%Uoc&I!6c0veAKL-oT-1ptXlL> zm{m=Gp+=1u*!d$Ao#f9{ zbT~?f2`~XBzyz286JP>NfC(@GCcp%k025#WOn?bgK>}2CxCn`hkhln`$`&E6S4`}gcWbXB8pCpBDkiXYx-*`x9~zx1TTc? zb0L&b(P@Bnq6xX8TobKZBnA{sbl)uZ%#l0hYNC08MvF(cc;<_p53Cd>O_(m1bFO~A zq};mBqSE{v4Z_T&K|fRQWCgyjn}uG!Q#$buhde zjO+p1_kz)VVC+pWu^;Sw3rxNZcD)Ox4uC!HZIFIW`uzNfC(@GCcp%k028Qq0`zm}=g`lgpF=-~w@YUevI*IQY(h36n~+V& zCS((`3E6~fLN;NQHDM7yhb!`@&x(Bd6^h_*lg6Fkg&;44GWt37QaaH@%Y&LY|Bxp7 z9@fOcMoqLmriqO!G|}{w6n0igYljlgC4or_I-?$SQ{wBCFhFOtno~EPCp2ox&vpEB zzA#Rinlti64RmzIo1xP-o}+`=ItDo(=He`!8R`nV(IWJ}X1U*J2WxaHBYVvw?~(7d z*4Rx_W3krjM$2p%EjCL^>N&PPk5@W6XT4vR-O(A!7I$>wNA}Gb)6r>ZF<+_P807$> zHs>fGXKYCa$0lW#PB48C>`j81kHP*fFxw3droh}$Fy8|fj)TK#(D5bc{0ek^4Z6Pp zJ>P=f)1dD=(0>LDoCAa3gP|Y5@Q+~RXR!SjF#0PP`wdL|4tD+lCZG07c0B{8o&|fJ z1Jlogy{o~@3t<0RuatDMFM28IP|{%nOn?b60Vco%m;e)C0!)AjFaajO1egF5U;>p- zfRYZEA#oWJmmyWzG9=!sn)j-%_`Rxm67nSENyyO=938rIkHZjFq9zZNLS{X zk0p3Xj=Cx5WAW6^&bcV){D;~(v!`%^&aUz7oQqaYx$T^c6mx7teXw(mIDRg>ozr}- zxSi8|;_U4cvvbUHT29O13E4R~Hfgb}2eV#qa3h%Wf%z?9!4D3%f{yK=^A*tbD(HR< z^t=vwcZ0q?pnoqI*arsR1Vj75@LOQyZLs}aFnR!ty$2@V2RlCmlOKUyhrrZfu;&Pv z{sip(6wG`E_8$YYpM!&6fVmT3{!7?7g|BGm7%IQ=qT=^wRU__Qk&!QUo*I}a|AbG-YDyn37JLchm@-S~)#vXsz+exjJKy(7~}unI!`jPJzSypyNBxc?NWy1Kr<) zo*zK(kD%{o(Eke<_!SKP28MnI!+(I0r(cq6e+G;`3&x%U6VHR4tHI<8VAonO^&;4_ z4oq(VdtU-Go523fV0J4wxDCv`4Cc3kg;&7gS3ySzbnbddN;O~@u>6S4`}glxi!H(?P;r&bYM)z4M^2I?)m5EQ`+VftJMWt4OjDV}Je zb*?6w7iePWZcQA&R}=C3G_msmO|(3yiSrL>6nS)!r%r5KF=FUk{iJ@_SsG+Fk7u6>~UP0+I+^u7i9-Uj{ef`J2I@I5f}J{bNGjC=&P z9|EI?!PpTn@d?=ZDVY2W>^cUfJ_mcg0MjSH-Y>z-S786wVD=ku@LMo<8q9wO7S4df z=Rn60=*)qxpFnpW^qdF11sFQM3p8{%Mu!P70Vco%m;e)C0!)AjFaajO1egF5U;<2l z2~G<0a_(9ofwLqmtROlK3a3E6~fLN+0rkWI)YWD~Lp*@SFDHer=DVG%=zYx1Yh zntU1-ir{aP&Rq>J1bHEp(a>2atrJZ&J*A0(RR&FTuaW3jFVVV1VnC6*ZkBs!@koPd zp){CQ3gg9x^d7I4bS~esq^!b@{JIh(oe5huMa7&&P;acIJRZf{u1B=Fkxx<5DP7X} zx3$({wAHFLa%+;?zss$`rl^x6>3F1{V{1LUQA#>PAHFkXl1@B-XlzNRDL6{fvD#*- zZJSIJ8KZ+^lQPR6pzrC8lKyAFz_VcRIWY7*7+wuVUI5$Ig3%Yj*g7z=0qlGUOl|_Z zHiN0HV9z!%{W93Q9n8D}_P+{dUjqkU2Xni@{2s8d7aZOPI@&>J2k44|?sq^>4D`l9 zUjp=h00uh2;K7Yj(g`Iua*R&pZ;{$$5S9JIAAkuk0Vco%m;e)C0!)AjFaajO1egF5 zU;<2_;t5dF;W8vHL*g=|DqDucyH)dU)fK;6HBUmGgggm3Jc7d`*o16CHX)mkO~@u> z6IQ$li%2>~MQ~L=SM>|(ExZsE!3$yfTnJ^9bm}-dr}VKw3M8fxP3Q6_a?s7?g3i^? zZ&2B(7or45rzEka6#3@z_tZngnRTs4Jzk=wW2v>7)Z3&1Q)@G;3-oP`db_@@Rp;qj zMV+m0Etj@4%40Uoidog6Yx=CMn(Eqh|HwO#KsP z)uMmGtZMoTHEP5tcNnCW^Mw9}t2G_$H`pwfbe$RM3cJxF^uK1g-)IMGbSfiz%_HxT z@3q$0O_t^b)_SW^2>WdL)NZo`q@iPL+q6k)I-l-6T6Rq*evMf=Ll-uy)_>?fh$?@O zyKb?X|G8}0zwF9GOCMdj?D3^)(}7W%j*yy8e9Kjujz#VJ(Qu`^W04`nJB(6ik;f)= zmLp*36EOTK82JorKL$oW2V-A=i4$PwmtgWMur!0h+n z;16K#M=<|0Soj4T{uOjw0G-32t7(&@+W~r3fnF!*YX5Y(h36n~+V&CS((`3E6~fLN+0rkWE-+O;|+J;lljsvoN2Ug(CRdq={qcnVdN~5$WdG z;-?|#M7EwSyP(r?w#GU~_i*YU1kJJxC^k9nOKnIT_^2nsdvKfqS1!LR5#LHmkb};!0*!3!ydJXJ(9Zc^Ad-s5uyE#`D!=ri;Wbg2nkOMoLY{=29l_ZVY(h36n~+V&CS((`2`k=&Mf4nl zBDkKP>-kL-TX-QTf)~Q{xe&^z=hRE*L=!C!YU2Dun&^9269*eL(e{`oHm=Y_(^Hxl zSY^=Y@cilUjMyn7PwJ*b*C}C!&R^!t$j>&ZPa1IG6;Nd=@})QE=iH)yerg8jv{lB> zNk}!v*8QeW`Z?>KU$xMBtM0_ygrPPwtcC~dhDB=2gQND|Z#OK)-ue1_=iWb8v>T#r>T4k$rQ<^mB#+W~*KQ|7XF^QS>oI@;h(Q_x%m~ES5{7 za*FPhB8xmUDYASACeMIf=fKqWV9yU=`bV($XE5^%*#9e-{S6%a9nAd!=AZUS7M=lz zp9LMwpwk7q)_`s|=xG7H9?-WQ^n1a;Mlk3DLtDVGAB?nu?Ex^l1B?a1#7?j?1SWTZ zU12cQ2KGdJQqW1i;p6-q3OY=H2`~XBzyz286JP>NfC(@GCcp%k025#WOrY`!P|%^E zLqUgv4h0?FE1gZqCS((`3E6~fLN+0rkWI)YWD~Lp*@RWrghd1$F3F!hOY$jLD1yID zDtEQK5afkWMnOjkIEwUdZq`J{98I*&)kO0GO$^ZQCiw(hcI zazvdvVJbn+F+hFEhu>Sy(prZ?Pc7v`I=spU1dO+`S(3b}NC&55380-T>88CbbjP!%;17I`@ z#?FF?L9lZOOy^Tpn3t;aBFf$DHH*J=7PS&xRc24D2UR3NfC(@G zCcp%k025#WOn?b60Vco%DxUxi9U3|`bZF?%(BZw(*@SFDHX)mkO~@u>6S4`}gls}K zA)Am*SY=IE#L(fI{OPkMpN54Z_}ip&XXJ$-FN880Itu6Jlsq%2PHbE;qUT)wq<+$b zoyEdX!VH~jDmnE~X=*3qQIG$YlFomgr1OL{h72}~xQE7NP$&%l$?>Sff)J z*=rtok9@DS#%{7SFR<2IjY8OG%cpjmB`5tHTT|;6De1I-e7NkA&cK=CV{}?~tr~lb zPD{%uNvBpGdfOJ`mFA8`?u>U7-zjAld2CW<`3x)^1BX8c9cj>c5_I)~?mp0y0llX{ zUq9#{00UVtcoqx|g5e=Bk^|d+0;72_b{6IQ$li%2>qMQ~L=SM{5zx9~zx1TTc?b0L&b(pf0Q6HPQdrHO%6 z22FIYk?2@2A(z_AeTv+5v)nUB?kHJi(WdGDxljzH_`WHR|p9wpN{| zZxyv7lXU(qzuYLt#?;8INpAlxw+5SXNzN&^q~nmKVQU$Lr1Mh8H&Z6*#Pf&7mULRz zjgoXMHmln5b#Wi3sJSC$7I|z^W(k0<9iTf1dUk@|5a`LX59gK8< z?NKoL4j7Api8$Dq0FxhpU7cX+AlQ=x(;tJqU0|jg>`#H&qu^i7FaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x zDoB8m4kaB*I+S!M>F}=UY(h36n~+V&CS((`3E6~fLN+0rkWI)YtgcC8(n#^t`yh(xSlV)k?}ipbZV@M zG;Pf4$(g!>V^%G?f@4-S{e>DeV$@xnS``Cxrq0pvNIl2a`p#DA=$zmFTiG3*_%#OU z3|-i)TK}O>^ikzRpSy0cn*X_M*}v?{LrWiBy6o|#YJAfuN5^WLr4GGfDyriY-6>rb zd2G^U83KJd(Ek${$b-T2V5k6wFMyF@u)S%kWYhu1R)Gm8*x3vwU0~N5Fy#h&TEMgi z>|GCLykP%EFzW*cw}3f6m~RCO0dRN+=y(-$z6QEp2i?0t&mPdb7xe7|{cnPS{b2B| ztNfC(@GCcp%k025#WOn?bgJ^?y9T!q9{NL+M{s-un~+V&CS((`3E6~f!iqOx5l5#+5nR;IMg6t3 zTX-SJIUKZGrp<*=Mn|WCvvW$F8#G@kOe=*^6UNK!oU5NNDYvh)s3bw>x+R;xKB&Od znN|8@A>K?!$5Lynt~ok;#&dMw?ig(4o}&|xdXBBl<(H1m7i-u3Z`NCNZ{{X=Gc&A) z2keGLYRiM8_TFzdEXLmX`g`Z%y?0n=-ZSsX<@4@ud}MiJ<6o*x$JrX|9Q|!|@@@86 zmI1{kr`xDatB>6=F-J#8M<@R2RdaN#YFpP;J1uIw!zf)Ad2G^UiGiUw7*2qZ55V?L zFnSP-CBekUU}qPY>;}71VCpE?(*vfDgS}}ma}w0t!Uo!j*7xB}~zofLv3GhO-y~ju_*y1h1Ca&{>TC@GoiT>gP-FYMH-u-2YIH?vYYU ztxc$kzBQRY3Fo4&0sL#J&#LuZBjq04XRM5LT!iyw!f6a6hzc0*?f`EnwOY_O^nV0NB3+%m%^1onS5m=68XGFgV->I`)CiH$m5a(ES$Zc^mY; z3;GU#{`bJZ`(W@xF!T`^J_JS%gY8Ga=qF(8Q!w!v*m(?sPV#dKItB_lOn?b60Vco% zm;e)C0!)AjFaajO1egF5U;<2_@(EDT;SwY+LE;jmDqDiYdsXvZ)fK;2HBUmGgggm3 zI)bAk*o16CHX)mkO~@u>6IQ$liwHVqMNF$@>Ckj^C?^_T_-_!7)>F3<4dps6(w!WPqw~AC`3~JV(KQXFp3*;+C zX^xoW)+Dz^{RPvXFPJX9U{Y)J7tHF(nffQpD%>HH&8%wr3pMJl@%)_0nV^%9YL2Zt zxJ~*wn@=ao?&mb0EAHoXpE!H_#QYqi%&PVOnTVf*Lz5m$8cd!9yL!P?AJ~%t)2G1R zelRlt_GiKDS#WR=%ngD099Z}X9L|G|UqR<@pzC+g{RilIdYh#88PNAE=zk6jJP!s} zgP|9|@LDkPBG|qTjBWs9FM)|oVCQBqxfSf%2BuyHd$w`O-Vl#KA^Q zv^}PYjVm9Pwt*BYSHl!>Wh3OcR3 z%vZ`dW(Yc43|HHQ067nSENyxbooEyO=WD~Lp*@SFDHX)m^ z;!Rk@(5dCf8AZ@CFjS{yaDB5w=#22U$yzI;p`&nePRUb)>ZHMRxuG-adHtjbJBx*> zgef}GmzmN>0}UPZHaWhg)@Hi2HLDneQ){bHZ`ZfA>O6g`TpFD-H6wH?Wawn1oMRhG zv`Rzg%UG`LhR(p5;v;lgcdZ(Ggw9alDnqAMZ8}s(7z50Yyg8VfuT)ccrzH;3bt6S4`}gls}KVU;yu5krS^3gr}9S~aw^6u}E&T3rZbG;|hnZcfQFgBFURyH^U+ z#fPkwQHIW_e=aGjup_^&bV)}#Y%4|W)I1jAi>y=p_z#_=Gh6p{rr6W@xBOD0tyZm( zTa(=WU2Y9FMSViwTGSQ#c80pbZnOyfuUWosw1YJ|m65&Xk@v{=T5IekOY;J2z11j$ zeYSjRw^?#h(XlnX__FkLVj7z@RJ-$@EmZuZX9~#@!>E1HR)2X#t)S;cmiA>VL zu}Poh1F+Bu4j%*^he78N(De!E{uK0l26~TyzRyAb7hvE782l0peFcWU1|#2q?cajY z(_rj7FmVR#JO?Jf2fKa%Q$K<|KZEICz}{cM%x_@-?_l;1aPaAuC3DY!`DekxbKvmv zpkoc_biXV;9aqcC^mHn}2BXO!D*J~&025#WOn?b60Vco%m;e)C0!)AjFaajO1eid@ z6QHNVl}KEP#Fa=@wi1aqtmX}?D}KXjo`gIJc@lDd1m{Pv3E6~fLN+0rkWI)YtauX^ z@pOzFR-y=M8iJYzF9co)(`g1%89kjk>7Hof_`RBl-=~S44``z0K~0>0NE3YzYvN#| zCfXj;=<-~2c@)vT#xP>)T>eDPx*4VEjQZydT08Z^P=cmYM9oF8A5 zES-o^cW-KS7pIt`bJ5Kyx1-~bdXBB-D>ypayn(ViI`M1F(iyt2S+)K{cYIVhKDSuS z|6I20Uw_FxEiI!Q9gEGXcAuM&ql06UF3UR5wE=X$1bQ}s-p!zIE9l<_23`h(+riK) zVE9!q@*3FwIvCvz#`b`TyF5~f=r92$zyz286JP>NfC(@GCcp%k z025#WOn?b6fyyU9M~99M9UVG4baZ&%bT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6INLh z7IAc_r%+F!r)8#|GM)4k{x;=rvy6^T1GE!ObgY+X-6AodXrlXOxo3{tF;^4K3p8px zy23PHDoiVdjHtM#GotJ)DYvh)s3bvWd=Y1Uxs{r_DHn9462v(=HC9E=xiPCJXG#^v zD4$s5)+D!@{z8qqXFNw|a^~oGq@H7I-5rpQPLm^Ac1Nejn}uHw{TZqF>sfMS!C zr8ccTcE`jV9jk4Y+7>ZQWR4DwO}Z>!gT8M-|F>Y^G#LC244nbP=fKGKVEYeX^hYrE zGnn`V?EDo>{swma4yOJ9d!7zRrk??Op9M3|f&I^e+123S3t(<7n12y0tOJKPfR2rz z(+9e?fNnqNX$8Fj(6NfC(@GCcp%k028Qq0(5k^3W=+bxC*JtRw40r)x2GG#cx;5laMDNPeM+Q;PePK zA)Am*$R=bHvI*IQ6>q{Kj*dYQR5Yk)n5by*LQn)Rgz0l3l+n?t=j@yb%?*;0&ZtM- zlsG#jOwpOxK${itYi8ZZL&L`>_N6!eei}OC8ai#`89I|QMJFKT99!FOFm%2?_i5P; zouO=TLnnS@-<&ZGo%p6vQ*?whbmGpdrs&L4n_e5G?O1FUHQr&ACW|~aX|n7ALwmvS zJ}~ko*uEc(z6HkK1{3dsod>|=dtlf5VCqA#=OZwE2<$x!W{!aUpMcpF&=*ocZQ=q3G^bUZ&Ea*QA1_r_45E#mV;h(@r9&A4kLnm6Gp~ERU zOn?b60Vco%m;e)C0!)AjFaajO1egF5U;<2_3KF29Lqms#4hr&7&=^&KYiBZ)38tkf17mfYIz~Z3!#jL4#(z{HZrJQ z^gTRE%^CI2C6i<7)Cpq=cFv8fHe#iiz+(m8d_#7QHgQ8}1(cxihXs}QJq-dgYHDpl zRrIYvy;I*B)!X!~>C)D$F3|s7qu#D>Yt?!BR#9i`Tg#>G47tS+9D|xQ=ueDl8$M~Y z8C8?qn&j4~zhL_F1=FP$OzOCH&aTSXIT0!5*y4w_OFJj=#dl@5bDGZ;w{yBroV|Tq zJIAgM-kF8LJBmffbenF!a(vD#Y3%eJy_TGVLz5EA?_l%~F!uCz$;2~Y=d)n)Ik4+_ zFtr-&c>zqX1$$ovGwZqaBvft+YIKnf`x71@XMfM2j~ofuAQJe1bTLX-Z1EE z1N{*&@CF!c2SXiTI0{DI0o!9>G!DiRVB!O?vva$YbCL(Qb9@fR=P&^#zyz286JP>N zfC(@GCcp%k025#WOn?b6fyyU9Ifu)SxcrFAkE(3>5${sXyHr>FF4a5UZ9DgyESqAUQNXB)BX#_ z@HEI{Unz_uk41PsdYaDVd&=eOXe9|c6L4$B6>;hy-~4r?*Lpr5@3|o{=sG^$@So}E z+&YDh&bSkFo{&F`K@NwxI7w%Qy25U>2>q{F?l;=O8lB3>Uh~L%)(io`|D^ zLz6DcVK8|F?D_;ueG2w`2BwdJy`O`bFTnm2VD?LJ@GCI)HJJYfEPM+Np9URS(0LYg z4TA0=(31nbKY_kH=synz3SjU87#aq{O*>vIBOn?b60Vco%m;e)C0!)AjFaajO1egF5U;-6SfQ}9w z9XdL6bm-{tX6bA~HX)mkO~@u>6S4`}gls}KA)Am*$R@0^CM@FUa8>^FS(Q)6LJ|CJ z(z-M9LXa0i866$vzu38%sBcs!9j0s9Dsq4Qq$xX#g(zW&&iG2r7(e_NFD=1wnXi5< z#!d=4a=6a;Lv*T2(77n)lv~hANjJyV_bmjSReLv;UC#kK}57B8lV7^ku zG23d@c))Pw&atQH;MkG1932z07D;u;Z87e5NuC^(T~Ac7ntY zX)t{f?Ck|JePDkEhEDbr4IKjw9VWm8m;e)C0!)AjFaajO1egF5U;<2l2`~XBQ27LC z=x_}Z*C25XQkAVi;?1ggv+9c9tePhwPePuAoE^d05o|&>A)Am*$R=bHvI#5RghdP; zlOnjJpG*49bX#~KD1sNl^tlkqXy`2DNfC(@GCcp%kK*bZFr$bMNo(?@7dOEyoI-8J9$R=bH zvI*IQY(h36n~+V&CS((`39GCLi+DO*nLmA2=F_uK1b>?}?+m;UV>HU zP3Mw2Gvd%p_LBxWI_hn@dsAJGPQ<9YIJLTqQ_RtsoLM?Kspr_596{;mbT%I|b`}p{0*5UH14=btrI^qhnFq-ZV~RmJW_h zx-4&kh5g|0TcG0|&=~_=anPLrJs*JHPSAG{^e4f<$6&Av40VIy6c{-Qw)cS1<6tZe zCQgE#yvLovtZ^d*gptnhrq!cnEMIL=fT2xaJT?Eeg~a@z|nC% z9i*c}M~4Y80Vco%m;e)C0!)AjFaajO1egF5U;<2l2~<7-Iyzj1#8pUKg;Zs$ka){# z-m<#lx2)z#$diyKAty+1f&`n8O~@u>6S4`}glxi!H(?P+r$!N6)XzozwX|D!At-_u z!t}Wi%IN4cKs(Vy$9jp@EfNEYCc1Byd*;X;b2ZVtK%>T^D@^mH!n9JDG-0~h(Ybuj zl5+bxi%Ju8G#E3MDSNEIcg>pOM+3fb>W=Z^cQpu$F5(DP(YFTmPJL^vNN-Vj(wJ8K(WcbQkzyEyJKR4j>TqGTM82rbZ~4^ zWLX8eoS?fI^teFp8qntk{Via?0|wWFAukx-2u6Hh`xY?j2V<>ZA^>*o0FyzmYbTfr zfjzsxbQtVy12Yk@{|zwP4i0vJxhR-_2Q0+E;W+4cA9Q{Qx;_Hkhd|F^(0e2(1s&ff z6m$%gUwl#VJGAl#U;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qq0u*#8=upt1phH22 zH%w;}vI*IQY(h36n~+V&CS((`3E6~fLN;NQHDM7!hfDIO&ysu!7K-3+lggdog&;44 zG738N9Go+up+VBn8TF`}5@Dx=5jvy2l{%O#)=7QL$Nx`B$5Lynt|>Zg<4HPC=u>tq z>cwd~Gt?D!qebX{%@%{rXa{R_DkFQ%Bkz&#wbs~8(v-2*>(b0@NHaEzNBTLoR{yI~ z(rNBHUv^1nC|g|8i67ZFXG}?_rDfC<9jk4YIutWaWQq=sP0B1O(03H{_ke-pU@#4a zPJ-cHFwzIMXTazwFxC$y2EfiNm^=%14T7m5uqOwme*$~+VCFp7UjVZgz`p-fRYZEA#oWJmmyWzG9=!tnm4Pi_|2+$67nSE zNyym|oE^a?WD~Lp*@SFDHX)m^;!RjY(lIN7tNOXB-$=cM7lI;qAxxhOp^TCa$L5qa zGN@kkJv_?J8THR4lOyZY2}22j&R8zal1VUW7EigFBV{0t(5bO1(w;G^Cud5@#wed) zypKt4HT{Jeb=P=y&P6My+;&btiaEBnv#@i%JKXYrSZ~$Uotvb(bJq;3;Q_m0k=pX$ zsJ-{w4U4gNzW(01c<&w7nfJ_la{0Xb8y{KT*l1TCTKeeHWsfgao6i-ubGlEQy?tVK zj*xav{B6UPc23cq7`D@P+9)j+d2G^RX$3<8FuVhd1i|*5U^E2Ac7cg7*x3dqBVgAX zV5%ML=>XGFu=gD>69fC>U^W2`egNh=!Tdq6kOYT61|6S(&QC$tXQ2BS==mJNfC(@G zCcp%kKoulFJBM}-?Ht-Uv~zg7bT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6INLh7O``< z9)J3*$ERJP2>v$d+nIPF$P1y2cFufhooM1DVMOL zm1O8lV3f^5$Ti}|AM$FP&kpko8*dNfC(@GCcp%kK*bZFr^A&KxTck7-2Gx%x@{q-i^gg|UQ5Iyd6o7*~o3KN|3n8~M`f`D_Eeajcm^ zlksZ94eL3#>Kczly+z;7P-p8?ZwzYIpg%FHZ42ZpMk$V%@d)4FTb*pqbPn?~t5m_F3{KTFqha9~nn zc?V3!z^*u$N`O5dfay-K_aK-_g8d(Z*)DLf8_cD^{86yb0}dYt9VbBNm!Rt_(ET;& z`3CfU3;Irj{_nuR88CPb41EuVe*hytg6%(p(OGVMQsa5|jw+5S{uF$uZOWPUhin0vQNl7`!*4G0=XU*E< zQ)cMI^M}SZbee+ZD<|WaWfV<&45ceNI5MfSxWM!pu-6S{TEKn}m|YJJdcoXAFz*8k zTfkvI=y(})ZUA)Am*$R=bHvI#5Rghdn`qawJbpKJOBZi zPZK*I&_v6FnmGTECi))M#KA^Qv^}O##_46APcIt&F z!O$s5t0_gjx%@r#&~R$6^{B^7lyoe$Hj{dr^j>P^coq!9skPOpx9i(lb)LRe}Tfu8R{?+>8wN6`N>82AMY{tAYE1H-?Akw3upr$ds_XTaFA zVB$Hj^La428ti%jOsxfbUIf$Yz}^jD<|VLy6PVo$4sHc=+ra$GAt~t;wud-JhmsBx zU;<2l2`~XBzyz286JP>NfC(@GCcp%k028Qu0+e(p=}^+4q(e!E_e^IKvI*IQY(h36 zn~+V&CS((`3E6~fLN;NQHDM7+hs*M(&$4_<7K-3+liHn`7lOPH$|&hHz&p`I$9jp@ zEfNEYCc1Byd*;X;b2ZVtK%>W_J52MX!z6_i4Nl5f18bI)Ti97tlA&`=kLFr_%|!P< z3LnQZ@>qoD)N`s%&)GAco-;W!baK+nu{He&J*TVxx&PC8tFFx41Z8H1)$oAbut;rr zaMa%W?S{qJJ70hAT)g)V>&$!RJ-K||{f&<-Z*2U_vgtTmW1XYFtxmqpKFczo*koU+ zO{`@wPH;E`I(CE3J)mnZ=-vl<-UPk-LEl@T|7|ev zE*LxjhTa3i?}L#K!S;{9=pit67)%@iJ3j%FpMqVVfvIC)&*xzJ3$XVDnE4Xy{|d~0 z4Gw+-=Dr2el6u}E&`dkQQ)N|@NH)ldKgRVAG>ZU~3DPe-nU*^n+ zLNlpP8tCU-8k`84ayPq?ZE$-*UkL;T>rk^tu7&Sh} zYO|`{PE+ZA4vtKEEI)&;UqJV-pyxNx`#b3S1N1-rx@6!PF!(GOdJYUf4@Opl?Jt1Q zwP5T;FtHBo+yEwD0=qVWsm)-|RxrH{?0p%`YzO;a0kf}ygRg?b zehydUPoEX}^eYs>-zJT_T3!h9LMWr3!+ANS%?hd)eGgyl=3KsK$>exCb;4AFn`2!m zru5N(ujA~Ts+*m&t1@nmM`}5?*0sB&o6{28Uv@XA`CM@~r~Aa&+b8Pg%u?G9Pr%K= zkx7T;ebDzI=>G@|90G%f!O#&f{0SKO6m0(tj2;7HpM!}nz|Iq3@=LJmD=_sn*z*mT z{ub;#4Q9Rr`_F*cbKv0jVD1Mn|07uV865rvbQD161<*ANx|?=MdK{p473gz<{$?=X z0)uOINjE3t-bFWuZVnS*0!)AjFaajO1egF5U;<2l2`~XBzyz286R3OwbaS}+h^vpd z`l!lQAMqB|yhU}zZ&A&YkS8HeLQag}#0WMan~+V&CS((`3E6}dZ^9yOj!*;_^K&tO z4XqYl2#Vl^FnunBGP*hQrE{W*gN>SKdrT7>S7@T?De3F1GH9ZEjYP+KiPkL=1B%>r zv)nUB?wG5I<^`GqoDV+ktd(si^6`Z`S|2vyOy2K7#TYvk-4A?&j)%?s={OF*hQwzeeA{bf+hBttbm%#Q-V01GW+X^POft@dd$?ag*D`4tXu;(=}{W{pY z8_eth`}cy`ec<4mU~WH{e+w+U4GzBxI^v)+0lGc_-JPK4Am~kkzK=nF7Z~UUgDEg{ z6b$!(k>g-{8h%dnB>fyi<=0+R{Kl;O0hj<2U;<2l2`~XBzyz286JP>NfC(@GCcp$L zo&fzE`Z@G-=;zSS;eFECgls}KA)Am*$R=bHvI*IQY(h36n~+UdWldPb&*6&v>9ZoA zeuX0V+oW-4L*RtSu6}C%+I-|kW)WtB2E36 zevbaCaiu8#qaOd~f1agtt8VpJ)Qj60>TIiGG1v@h)}TL;ipT=_4x`Pen&j3bw?_R1 z)1NPxF1=tdBew(+LuJRv_|uvyfL+ZpN#yU`-_zh=4L zXa{R_DkFQ%Bkz&#wbs~8(qysL>r%@lOD7@)9b5eM-O|!YZ8%+aOK0Fr@hLj3yH<@o zMW?0ZDoe+xHr;cjrGw;K9=wL7gCmnR%U59ZYcTc=nD`d#JPjtl1G~C9Q+N;{SM~;01Hp=mK=TtbT~n0Gw5=G?lqvt4SHKZp9l1>2LoO( zxDgEb!0;9@;s@JX!Ds-C?En)&uyf~bY3U?GyE#RNmJSnO0!)AjFaajO1egF5U;<2l z2`~XBzyz286R3Owv~;)*iR+NK4ynr4A@OF_yjgX{Z&uBdkS8HeLe7rh>5EQ`+VftJMWwdk_a&k_|Q-c=DsX;4+;o?J1 z=DJ$cxqQ!(vPwJh>q-!HN}6f&DAbzrm@|vY^#4rg=V_p#Q?jD-Z{6Xkl~NC^pIY_r zx|*Y?D=sNI7q{h7bQ03ev336nMQ7jBe@vO86VD$SThVDRCK85P|=~HLq&%-O=lCb z3E6~fLN+0rkWI)YWD~Lp*@SFDHer=DVG%`#i}I(>qI@bAir{aP(w%`9g1ivQsOZ#5 z?L-sD@6|;7K27X=KoczwYU2Dun&^9269*eL(e{`|jpw4qqloS`mkgZ|Rp;UpIqK$e zJ?HA@Hz@4L$uT7uI@jaf=)#M6rMT|L^?d1sl$=p-rlB+L5S_?)Lv$v`&`C)-$JX~z zSQ$##Z0MW^z2NNqBR%KqUGzyz286JP>NfC(@GCcp%k025#WOn?b60VYuK z1gPk65fT?6aS>9LEkfeWs(G{Oir=i7Cm~Nlo`jqo!PyaPLN+0rkWI)YWD~LpE8c`f z6rCDHa7{ng^w&~u;e{ZFa8PcUHWxw}6`clHCz|M3FVVV1VnESE_sw$89JynzCYl#$ zw0LxbX}&a=q%NXih)Z!%=i>e)<<@oNteO)1oa=UM{`#Q2Kvs&{(jN=)W-2EJ&v=T?6Y>KFo8{6lopKG)$w)iL zHnd@nRCInm9xl70({Z-OI!8Bg>ZFNdpJf?PY_hV{rq##pn3$qdD-XT3$v6Q;2ZttA zmP26nFgSPw%zXmpKLrb)fy2i@$8pe^23;pXcQ5Ga1HBp0cMA0PgMk4sm<2;;!SEm$ z83NmLVDu+2mIo8(!OjAhya09$gQ=!Hl06PEy$bAgf|+Kp-vwsZfP-!@*8=7}d!(XM zSigsg4iz0Hzyz286JP>NfC(@GCcp%k025#WOn?b60VYuS1gPjx(V?P4MTd$G@0iXe zWD~Lp*@SFDHX)mkO~@u>6S4`}glxhpYr-Om4j1LqPNAkHXs0NG7s9l<5Xz|N)N^vq zgr){vt)$dViLz6IqcZ`wrW6ZjG5j0L7>^}*m4Z&y2|8`#2|ANALnkNQ99z>V2s%9< zbe3Jv8OjzHbmB+$%^6eBX*w`!hK|K%RhvGXh@gW*lOoGYU||zDycu-(L1!!I3V`k% zpeG1=cY?kU=-&kf!eFos3`M~38(^dzZ0`W0Q84xnn23R$aWI(xyFLI@onX&FFr5T@ zKL#^hV1GB5O@V_)!CViRKMoes;P6S%@fGO&8iJ1N8wxsx$}hgC_zhb5126$5zyz28 z6JP>NfC(@GCcp%k025#WOn?bgJOK(iT!O?UNL+$cWlNBFt7_h=y5hI0=1IttkS8H0 zM{sfkn~+V&CS((`3E6~f!iqOx5kbeSh-tMfecFt`pfW;bgx97b=uR20??Nb}pu?d# zr406=>(I_V3!L_tpR)7V7dkD^?;f6 zV80j4ZUhH?U~UVT_k)F2a5w-uUICr2g09y<_v@f%H|X8dCjA`W-Zl=egTKoN|NdYg$CLUmpUW%P6AOYcMz2OBlf_LwF% zuFyo&Q_|U4WzaZrEecFG)Af5(! z>??&Bc`U;7u?&@RDLYz8j?QR-W|U7;#(%DWnlnoiv&KIB$#}H^flu9$Ui_{GVYSvu z5l5(szBQRY3Fo4z$&+M3k``oC+`+x2a&I#1s!>TG>$QE%0^GvvP&i@|14vj+W% zQEkH~jW(ldl3SD98ub@Uf4*S4^nyvP(O)pDCui!PFsl~*6J}M@U#L+lG(g8A-5gtM z(_Sg)Y}}GByP$Ke0Xj{Ym^!AQ)4FTa0G(M7bdF9$(7~Zek>w50*ADtSz(5oXz5|A0 zU^os&5@7oWV6+pA9Rw3eu=8Uu*#&lWgQ*nQa}-SXfW61TOd9Mz31)l2!9FmT0rRK8 zLO(b>06NZq&U2vad(iy@==l-!{tWtl0sX&%f#1O3?+|oCe^Aij039a41egF5U;<2l z2`~XBzyz286JP>NfC(@GCQt*M;bx+; zQJoZ+M(mucpVUv9ud`T~N*JOeb(txBG*Hl?pd*BRwxxN2-DU|$H^R_ueqh zFx<(^zVQ5de&=}3ndP`I&fv_sV3;{GUNEyeUO2*xXBc(%?4d@C5TKNR5dzj4r9#9~ z1B4nNlmGz&gc6`&fKsE@8laY-5hB#{J_~j?l+dP^0ftS_68 zO~@u>6S4`}gk!7;^9VY0Q|P8p(ki2*r3hXKdb#8{fy}InRgwglm{0I^XNQg;)DKpqjYYM(6QVZsWVEW zbpEN&+9{XAcQBo&T>Z1$8f=RChQ77j*-lUw*o_vU|2NCmjdrk1r_#UI-2Wc=UTc}% zBn=m9r7pYNF?9k`(y_I!4@grdd2Pd}nL4f6!-JbTZR_tdb&P83bN5O)W@+lkfnoRV z>^F6AXwqnD149WgdaUhR2SIQ4W`pz&t)*v17>@{ zTn6m>33U7dI&XlkU%}d2p!+w_^HxC8`!-nr4(NLqY+M5Rmx4{p!N3YIxDpJl0>kft zk@vyq+JH25V(S7lb!h4^0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqPq&YmvAX ziEEL@*jgmsv6^?RF8Lj+c@pv@M z=21l3GDE+sbN3TD>}GHu<{`bEN?|BK)X5{~%(+XXSqPD)n2))mXy)|$KUGw8tcp}< z%ExLhYRyF;FGBse-6`XQi!7)lTXP~09;Etkmds{3;CnD_}TU#26&d!Uw zi>~OjK46$m@6C0p^{=`sq+%G)V^;Iu=gu&W77H-kL^FtZKJ2Ekkl*cSpFyFuq3(6tw=-3Pi4 zfS!Y(_b^z01oRyP8;^tj6JXOxFmM_So&iH&f#GvtQ|r025#WOn?b60Vco%m;e)C0!)AjFaajO1eid{6QH9*M~99M9UVG4 zymLC6kWI)YWD~Lp*@SFDHX)mkO~@u>6S4`%SQF-Pbf~9LPobw}rk*mM^c4OzW6PH9DtX(2-C6ppMRDsl}ArMrVx9mZ2P-e~-~AuA`HXdXBBLWvg^_ z)*m@tbVsM3AD-z5}}61#6dp?xmn-Ip|#h)~^J8tH8$h zK>zz-(^@dF4h(((hCT$tAAymN!RQ7s_9+Z-Yx`E|=%jaS zNp!5mQ(cvm2u0rA}q%pP%iMOlf z?W#+DyK0_#=+Gcr?jQc}*b^*XmnL+4iTO3@9S-gJILr}fy@DT5k1t!oEN(Sa$|xy*R)Xq~~Q z=-}9-$r1w7yTG0>m}v#G5iqwO?2Ceq!=UpB=sE`09tYhgK+j3gdm5}i1Ny!K8_$9M z3t-blFz_`P{00nt2Zp}~BR_!AOJM8@82=GWTm{>&fytl2j$gpk4Y2E1FntT``3=mx zwM{bn_BLtg1;waA)Am*$R=bHvI*IQY(h36n~+V&CLCi;n8(oJn*8yzCZC3d zBKX&&b63s_L0$+&G;}yNr?8Phm7?>d0cy^GFXxVosZ$}01=u+cui97^ieWq!;LWsi zH0S4RE{&a&kz$Um<}~b_4&OWfi}i7R*v%ty z*v-=utcDlthFNOeiv#vPZ#T@r-s$>#r{cX&SttH&+MBOUd%pVR*Q%@S%ItZs%$xuE zJazGv{B};;xyw%u&(5*ftZM6tVc9u2Hfgae26Ha3ZyD%V0XkQLu2o>|d!YM$(6biw zt^?~o0DT{VjUR#jkHMx5VBk|QxCsn>28KTeBU`}e7hr5F7~c*ic7W|)g2|m=$8IpS z2khDlruTt82f)lhFnbux9Rd4}fsQuNnSh<+Iz>CjKs$#CFaajO1egF5U;<2l2`~XB zzyz286JP>NfC-d70oplSf5i1iTz@pi)*taU)x1r0$!}B5laMDNPeM+O;M52cdg)n|Dgd*BG)1`Hyi9^+zXns`_YZhpt z=3mm-S!B>e+cJrk)e;TsCAt;4>ruI9irg_(6N_hPqW2k1oPAakt6|irtLq4YCJ(P6DoTM}42%R^iIb@J?WNwer znV>GP8!bZrZ@+ny;f`3hFcSc?a@_9JtfwG)a?19-A~- zHh|tw!TL?0?=!IRbI`v9Z2AHWYz2ec!O#vc{3RIK2}XB=u{~gXFPPW|wjThK2f>cR zVCo3ibqq`&2YXI{nUi4lG?+UB_I(99&V$Yl(A5dnra!`F*;0u2`~XBzyz286JP>NfC(@GCcp%k025#WOkfNoKtqRXkhlhkYmmm+8YJGW zns=)%`Q55{67nSENyy<593H_YWD~Lp*@SFDHX)m^{ioP|df6=!_^$C4zy0bN_GxV>^)F<_A zxjIeXD(Yl?Yq_(XAh(K~SYuGr2K|XqZN_&RZAR53wMxl7{etPv3nsNpf5EJt zpQwL_S+(fjVOBN$g)+5&n9e`-Hh3f|17r#o1*@En9l8Ov4-jRq@`nP-V>Cn z&RbnaM@`jf%^n_H)oI&qwpQpyER(=TA)1{nMm4BZ05zk!jrf|Aj9|W6Pz(5EL z?g~m#Cln5HlnzJfFaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x(kDPshoTNe9f~>> zb$Hu!HX)mkO~@u>6S4`}gls}KA)Am*$R=bHjO11hs6DCrV;VY-Njaw6Hadn*z^I!z z<+_Pul(*lV{2W$LmE1%6Ws0Gsb+bm5X3f)&S$9NmI@p*4{Wrm;J}^+TLo(<9LyN$$6O1edqplrN(TOeFK}DzZ%P(pS zqO||`3orpDzyz286JP>NfC(@GCcp%k025#WOn?cLJOL^?T!h3$NL++8#ug#*messv zb;)m8&6AKPAx}b1kl+LfHX)mkO~@u>6S4`}ge7mnJc>@4BDkiXYx>J6x9~zx1TTc~ zb0HK_(W!!UqKTH(5)JDmx)n{dJu3H1kvpbpV(|=(7LRT)O_v7KLSfQ`X>jjlZn1Tp zS%vvI+EAZ+zc}6h`Z@PZ#xdH=YUhdp<8lTUbZ~4^WLW{mR)XJ_OSrfju9CnGIm}Q!uv)?E4IKG=a`$&=mk{w}I{;=xG7HA+UZI z=nI35t)M>wHth!kQ80K248_3kQ7{q*qitX;0me^(iFUC4ECiiol7fzbf({d40!)Aj zFaajO1egF5U;<2l2`~XBzyz286DWNG6m%%)P|%^ELqUi4OJ@_Z3E6~fLN+0rkWI)Y zWD~Lp*@SFDHsKg+!aRZwm*kJ1CHWLA6v4kHl{>);L0$+&6m%*%H)mKggQTG|;88Iu z!cGAbbfhnH&!Ylgnn(089Y3FvPC=5+_&^w|C=oa zo6!!I=~Vjnn)}})-)k+io1`gYt< zpyMadnFU?f!P*??z6pBzKyOWpWW59QEdm>zpnoyg%sPVFxdch_`uXBU{_;{lyuS?TR26h^b0Ubev4NA0!)AjFaajO1egF5U;<2l z2`~XBzyz286JP=*Pk@pRmmzT(5|<&3v1LfSSv7A~UGke%^CaX+$diz>BRD&PO~@u> z6S4`}gls}KVab~?kECN(1XuNQRlkvX3oisk@In|r7eWyw9gfW@Y-CWS=zM8_oipIe zxg#U%R0u->g3e$r&fF0&Y37W&nj>YPDtxGQlpj^Xs@pX}RrIYv{foXes!!y8`c`{Rdxm+ zp@Un3z+!=%x(p9+rhpapd$o2cY&@jSlbG^BcNwL=#7H)hd^HpY&;73 z<6u)87)XG@Q(&ka44(xfNicdIjCFwVPB4)I+rI^qU0_Eym`a0Pm%(%o*wYJUGGO*6 z*g3f@?HrEKVFFBm2`~XBzyz286JP>NfC(@GCcp%k025#WV;}+AIkaO|(9z{TGblsglROP#F3jv+#W2M4h|$6ie9A3NmztG0J8pJQvHuKNS3sVSH{; z-b_iS%&N!{H)i$xL@5Xv6!;*AxY*J>q0p@-M`)+}bnlB}t4$!p-taXC!#h}LpdY6Iq zwVy zP3OTt2N>)GLn$!)Eg0znqupRE4aP5ni5{@M7ffcrj-S9(7VNqXrgLD=O)%33W@|!{ zIS1Ic2z0y)I+uX1rC{xH(7gimtPDv*$Ga*-L#OnsFKP^;wEy@EFaajO1egF5U;<2l z2`~XBzyz286JP>NfC-d50UA0qbZF?%(4nEjTc)!K*@SFDHX)mkO~@u>6S4`}gls}K zA)9cFHDMk@himf3&zgK17K-3slg?ckF9dlZ6w%O`$D^zeluL2+`)A$1Hp~sgTDjcpJy)SjufC^$9u3rd*D)nW1mX)F<_A zxjIeXD(Yl?Yq_(XAh(!>V^Gru{fSX+#&;TRM%5&@Cb>20FPQ%Qg6Yl+Cbe)$=b!qI zESKMbA6c&cS#AwBMI9MQ$1VLFTSE$x&c?IXMorRb%^n_H(y3`2An6Dx=`?(M-xwW> zT6fEEueoED19#e-CMmPXW0NwA2lTE1>%E|FJ=j@^Ce9j zs@6pFs~Sa~+aix5+LjsmHJ!Vk$U!%E3p)2de@JDgQiuW^or1)gLgbsf-%|+@M?B~u zf3XqLbSh#0Ja9BnC0@EmEgFR{<=`MGxs=<4s_0vT`WJm`ykqE$EmJ38)Qy~S^$oc- zN<(L$se}Cnn?--!V4I*Wup2Ex|8JK2jdrk1r_#UI-2Wc=UTc}%WLZ4JT4^;3VV^9Y z+HDq}lyq#(%{!&3vt<9$D(mC=i;u_`pPpbfykIxXQtMtEu=jbpVGj0A*WWu8?|sTT z@o&@Kd~MqE)i1wRU2Rup&wFLw{MYBHtq&Nd(|dEBYW=JJ{VKlyF{}CS^XLELKi^oh z$85FhuTR1+s#t{dpqeM#>*QEu9IY(_?45p52ZttwmTy6S7ueJd2GU^gG8pOs!@Xc6 z14e%WV_7hM9ZclF_M2d`5A3MfDVcJBU5mi96YNgP-q|qE2YbPEOM){UVH#->Q|r025#W zOn?b60Vco%m;e)C0!)AjFaajO1eid{6QHO=QHP=qMIDMdymLC6kWI)YWD~Lp*@SFD zHX)mkO~@u>6S4`%SQF+Eb+|Nt{4C9=xo3*pF;x?bXJ~YJbdPDe^q8ctqCri$Gg2q-lexu~c4if1={(S-d5~i>-2JcI zd*HKhm=6?bM(CxgKc=7axbE|e&Lo{JL-{!)Gf5{P)f`)EAN-v3mXC|>=d@fdvrf@9 zn+j+)lPukeP4<;qv-I>+!}D`6uShyUkLf?`PxxUL%iw+v4o!M20Wh=;38h? zd;@yF1M9yBeLsMWmq7m&u<1uIa1{(*14BQ9;a|YW4KVsE{G8Y=`Z*k=!vvTB6JP>N zfC(@GCcp%k025#WOn?b60Vco%#y|q}bGQPDE0DMXX^gEv;{B?5zv`0TubL+zPePuA z93R2)5o|&>A)Am*$R=bHvI$Gxgn9fNgCe+~p9}g;G+THfD1sNl__+{@=;u^&c+Ri} z2i-5VRNSkr+^_5u#n&ldjLu-r(EY#F!|$o0pi}SwozWC@noA?-M5LQzYdgA23OYyD z?JT;W)0@sO=(HZ&I%QBnr`Kn`SH>~h%&Js-?%g@Kp@UKCTMs%u1f3s&u8+am4WRo|(6b5jeg@Wm4*Ir$ zjbDKNtzgr3Ft7s*ehG$lg5lj@WDgkK3&!?=@dIGuAlQC*mo#*eM|N?74kzd^0Vco% zm;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oh7JuK8agy|c(-&mA)Am*$R=bHvI*IQ zY(h36n~+V&CS((iu_nx8=x|N`_*s)r!$J}KYtp$Z=Y=3Igd!R`9GO$txS&eW`O^J% z&fRIp&38R3BCNIF8S;l-o>qEFH5tLm!hG{L!?JU5Y|>(hgUL3qBLSvP zfnDuj`YhO!1T*KsYzLU@1p88;<2%s#J?Q!Yti1%fuYjH(LGM+t{u=1}8EpIo^xpuR zegy-!z~FCS=&jw7;kUuaJ7Dx(Ft!AYF9j3J!S)qkawXWY3QWBRcD=t_+Bxa9yJ_c? ze&t1tL6r6%e*q@I1egF5U;<2l2`~XBzyz286JP>NfC(^xk|#hrhwG2H{)p?3#@PBJ z-lm$jsV@0#s(BLfB;-lRsS%tS!6sxAvI*IQY(h36o3P|fn8(f$ir{j7F6S?!*TM@y z5xfw_&xKG#J7>DIPBd|-S`*E$YGTa-P1O8L8as;&nrK@l(Xv`XPMe^`)AneUXqmE5 znED^H@LUY?D3-9J4b#w>1$U-=q#v)~BWUQ1IYXyJ^K(*C&aw4&!_ZlNEnRd&=Rxyx zYEof!P(!ELGhlvBxy_<>ZZHmKeh!XJnk?(VbUoP90A_q(_7gDI2=;9R9iM^D&q3D~ zu=Weky%qFq2faJM`Y%D>POxz|=-&f2?F9q-z~BKebPx<51|vtn=rJ&M9E_g;6DPs; z(_r!p*zpyZItO-L0Mi%2p0B~oH(>TV7&^J{Y3LYe=r92$zyz286JP>NfC(@GCcp%k z025#WOn?b6fzl^HLx+YA4ILUfG<0~cbT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6OOSa z%wyVCsM(gs)@xjG|~HvCeA*qiPq;dvGD~> z)V-*Q>$5dlJi5VDAq}Q}J?H)>l_MtX%n^nHX6QUn$*CMMm8Npk4ISwORpCReqx`56 zR(%YPP!)Y^Q2(NDjd#Z9jIA*`LrOYt$Y0(dO_c@uc7nRVZnOyfzgg}#+QBlNO8;JS z|9j+ntz~wTW$_GarPU~eeX@LNw^=gM&#~3)2un$)r^E5TS|8V6d_=zZ^aQKn1-oIE zTKD3Bz0cbXbFg>1{@$s0?^D)^f1CE^Ytx>ue)+ZPYP&Lf-YfIwzdldxzLbBAPQ&I! zgOAavs~aHcSZr3cH9agz2gfF5mNb~V4EFVaj;o;a8tD2Nto;Ra-vB+og5FzT{coV} zt*~U{+o1m)u<2beumlV)1w+fh@Cq=p5{#|_WAB0S_rb(kuzekv`~d9u5KMgpc6|({ zH-J5#f|*TV_A@Z|IoP)abOb=>wy>0RT){BM=#+i|M#=Bd%3pv9FaajO1egF5U;<2l z2`~XBzyz286JP>NpyUZq(%~{BE<@rnq%pP(iFd2!-KtA|w`!h*JPCOca(D!XN3aRm zgls}KA)Am*$R;d#6Xub0Op4&Dey-{_Q*Ysgpa@5y5G~ed(YgWIy>^?3h;Ca8fx=O@g$d7kYx(`&?3H#l1`abkrItrJwH)8Hb(ix zBDW^F)$|w2)P70lpZXih)iV7JNfC(@GCcp%k025#W zOn?b60Vco%N}m8F9ZEWsbSUXi(&1gx*@SFDHX)mkO~@u>6S4`}gls}KA)Am*IL4YV zkEFw8`Qv94kbX??&To!*=4RO?^$K|Cr3@jPZV|9$@afBa_xPwUzNj*ft%v-ZA0I##vw%z&L% zIdZ4XX_79BJT~dFybF4lfb~m3-*T{V1?XQ1Hmw2!?}5Sh!O&VTybg?f07gFqV;_O> zkHN$Su>Dgoxe4s}3`~6vc5MOEUw}PZ!OV6ry93O93HI#-9bwSf3c4a-?S9Z51wDsA zZw#zI3i{$;V;dYDe}ayVq4eu7N`9AC{sK&Z2`~XBzyz286JP>NfC(@GCcp%k025#W zB~O5k4p$*@6%tn=jj>foyk9l%S6%Y^Rr4g|Nyw9s<0Cjef=$RKWD~Lp*@SFDHetz| zFpr~CrU)+T=c4{{+AX{g7&T!WAm|MEa&EDGommA5IuGO7-1%MwvW_VHF%xetf}^7hbF|M) zctxZAkTR1rUdnAkRrIYv{foXes!!iMIM^8Sx$rgGhowK zVBj1Wya0wSg5j^h$Twj0J23V=82Cn>QUDMfwY(h36 zn~+V&CS((`3E6~fLN+0rkWDzonlO*0!*%%-RA_1m3Mz`=g)puzgd$oxl~O&?MBR&; zxISAGoiAzPP_-tSU)98#1)8Y&m(+F^Nq2`T&mE0PYB~cR6{BM96fj3;7_rfbaQO2 zKSI#SEKe0((CJO*7j#;WZJjcxpi{HQY`sU&G1|;(+y9%_#mT!~k=_1x$Pawr>TK+rf?^lfLVxaRV z=!%22ZJ;{=dQO4fcCh{|=u3i)=Rto5*whIIQef~~Fw_NxyTM2rj9!MI6YHU%!!bHc zfC(@GCcp%k025#WOn?b60Vco%m;e)C0!&~GBtSujOOUt(iA#{i*b*e(t(teMF8ST6 zc@pv@LsA zcOeu}(Ba&i!e$0lN_pvSJLUf8y02mwFjD94FMr>7oeCiekaGqTY3~13hr4GwR2=hw zM+NrK&yj<5h900Zntsmaq5PbG56~&DpA(U4j;(F?Ug_tYym6!Ge$L`6`Td->bC;hS z($9HJAE5Im4A8My%(im1`9niq7bovdDYD38lOoGiFm?@${|qL60o!kY$zQ>aTVU!p zuo_D~^yI^(+m|F_=Ee9QSpwkVyR)e)3(7guqctP)au)ZGjHGqvi(Ekb8 z)CdMPf6JP>N zfC(@GCcp%k025#WOn?b60Vco%m_W%BprAuRhk_0T9SS6S4`}gls}K zA)Am*$R=bHvI)mn6Xp?gxc>cju79UGpa@1szs%_lJd-Jtv&sV?v zT6MKunLY27dGlYNr#@(cPE9JT4r=H$Y&PHP-IT-7k)w<5(RQphtJ>x?<=rVw7I|#a zWZ45I_ktb!z|;Y->mZmu4E7uWGsnQ}aWHoR>^liM+Ck@8(3J#h&x7s`(9;QeQ(*nK zpsx#T><0a5u<0@w=mCShU?>BIe*z;}FnS$~<-qt&FwqCL*F+?f4zObpm~w($izCv| zNxLF6bZF=>0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oYmm4GiEEI?*cv3> zshW4HF8Q6Rc@pv@L#cwRYT!p;0f_GuYS!`kTR~02tT?27_Rz1q_G4$SyD%24k&YJOU>6 zgY8i;c?j%?fvKZlR~$^YfjtQ@a|+D1!_di{rJ-Xe{pyR7-=LMh025#WOn?b60Vco% zm;e)C0!)AjFaajO1eid{6QH3(Lx+YA4ILUfyk9z-kWI)YWD~Lp*@SFDHX)mkO~@u> z6S4`%SQF+kbSS4#PNAh$MoUW(yb#9Kg-}F8XQs4HG*R;}O>{3ZXrgVIM9XT4hV>HN zirn?6+(U~;8cZ{#!L(4AZa-wL3@~&Cd^xwM!jAm7!X+JPKQ0uO(Rs|o2U$n?Q8ntH zC=XD6sP5wkRnfNw^)LF?IJP~Vf68CdD96T>$*oCl|17r#o1(s86ol5^+bN_qfd#z=5lazC;mHGgjJDyHPDmu2Bjr*jhlijs<)I6Qm?BT&Z zowoG@JROT1np5-3usj_coAg=Efw>D{-$l@o0-fK2t}d{)8+4~Z&t=ft1J?I~z6{v- z6X?%^P1nId4h-G|Lw#VlW}jrl0Y(>rF((*b3?^J)`!Xu$tk+W_FXgUMFd`N4j zQWy%*bn>VcQ+jzj7>wz@1FoiiK1 zD!QZ7`hZzFy*JmX*1zhqkSe{QrysML|2}{IKkUlvd9Tcy|N1<&u5N&%W3^3EJ9nD$ zj?KxtQ@Skj*rdzi2VI-N+9uH540-~fcN8z|ej$90enX zz-SDN9R=fYFwq9KC&1(>u%jJJodvs+VER1R(*b5W!E6f5eGB$=fsP+Q=Oxf}1+4uM zbYBHM*Wl=Qf2O0uSvpLB2`~XBzyz286JP>NfC(@GCcp%k025#WOkfNoKu3p;4jmmj zI&^e+-*h%1n~+V&CS((`3E6~fLN+0rkWI)YWD}0DCd}jLP*0(rLQl&~J!L%UDg0~7 zzh)5~ohpvcDRgwubg}V;g+fjk%CG74D?4+G?d!}cNYEL2p3QV9HIs_?sK8Gf`g4o> z9?sDzvntZSF{|e%N)^W_pIGG9B)6LWLYcZ{C`V^x=IFSko?~m+zh62!Evv5<-O*{e zTxOl3t2h->#j#JabSpMF@k^~)dits1IXXf*It_8dKpkh2+8oF)?nswK9-DMou7lnj zSbr1r^?{8w`z8GjuxSw(aDu_bV8{iAmw}O5Fj@!3++ch)nDBt@Yrv!z>{t(`>cOrC zFzo|-J^?e0V0I&z^MiexLB|%*`330O3f681-8(?fm!NkiSic+e?ExG2?w5{^f8T!2 z(JB2sQfmyNwEy@EFaajO1egF5U;<2l2`~XBzyz286JP>NfC-d50XjNdg~U}zT!l2o zRw40r)x2GG$!}N9laMDNPeM+Q;PePKA)Am*$R=bHvI*IQC2zt!j*dYQR5Yk)n5by* zLQn)RgzR!~u_1T)}d`T0Bsx{I4swUPf&_vC@G||0CYAJGlj3x?| zS+sHdGo_?6;88Iu&Q1YSbcSc&EP!7l9_FFp+lOc9EO=;3NGp*RBrTY7n@|;fYf%59 zZ;k2``qp%3YgT9IUze#*>f3U4n!Z)k$@E1-lwb+|2FN-*QPyR{qk$o)&CiW)0@t3=(HZ&I%QBpr**B_ zYS(}MB&a=#MaXoUpT1YnnS{wYryeBe;LxPV5(WK-z@``&I0^>iV5kiYC&0)lFxn2r z&VunIm^cr%cYw)GupjnEVpyOxI`3vZ}0oMKsx^ID= z-$3tM2PEs?27T{NfC(^x(kDPchk_0T9SS-Wba=ybHX)mkO~@u>6S4`}gls}KA)Am*$R=bH zj z`>ZBfpBvAz&2WyrQp!sM%$xyV&K((1r$QJDkaHg1v@tFe!+0#fn}=rvJ^cHt{+M#k zjC&0x9->}duw0WiA_%mu-|7SOR1bnXUS zd%)VgpnD(aIRJVOg7t?%-x09!80bF^Hk|+iC&A!pFmwhCe+5R)fzb<)b7B`M=NL-A z@S^0mX5}xy1egF5U;<2l2`~XBzyz286JP>NfC(@GCQ$MODCcnb5tkov`Oz3#e#Dzp z^Cs0LzezPuLY{;?2{|)@Gb7l9Y(h36n~+V&CS((qyb1HjIYJR!&Ck{RWz<@DAt-_u z!uYumiYVty=fs?Xrv+6>f$46AWflyTfx~p}-cu}JM=MCs8HQVP_fh11E|s8#!aU?- zI^IJ^M-J8*dVtPoIyyrR(0M}&Lk62gy*)^0g1W$NvNI$a<%s} z!@WBPpP++dlQK&RjC~8nyTC*@*q#QHm%)x6Fx3loWx(`LU{4mzTnDo`Fn1H|>jNEc zMJ1hYgRXbL+IK6S4`}gls}KA)Am*$R=bHvI*IQW2_1DNIG1WKYo_wQ?gJ5 z|C-e9jJy!!g-}FEN8#k0f~N*mh&2oHDlNBnR*smnGe?L5rsxdq)C?)zn>1OP!DImJ*aoJ8U{?#64uL(pz)Tp-wt~3`*tZ{a90Z+*LDvzm z_890s4th?2-jiVcY0!5DZ2SuJp97mNfPsr(@M|#i4H*6ojC>D9e*j~b!1xs~@gvxN z6--_OJAMXJzkppgVCbZOrJ=(yI!u5GFaajO1egF5U;<2l2`~XBzyz286JP>NU<@Qc zLx*dSxCV)9kjB^=B;Ku>cdIV>-Ku#K@+9O*$l(zj9>FGL6S4`}gls}KA)Bz|O_;~f zF)4ye`njavOt*y>f+Bb!jGqgkh=$HgX`N`I=3koVUS!Zj+cJrk)e;TsCAt;4>ruI9 ziri7K$)b(ZpE*F4+T%? z>5N@Z=buszGRmAhgjJ}`3t%pN=_J)PX)gPf#O`n#mo7({9R@fTnMOn?b60Vco%m;e)C0!)Aj zFaajO1egF5D0u?(bm-~O)1jwBPltC+XA`ms*@SFDHX)mkO~@u>6S4`}gls}K;TUVe zJf03$=8vD1`SdIl!M`TWI|DBSc_9?h)2Wc|i6+iItBKa7G?`h-3s$8=|F zR%hs6m#I(c+j4c9zE#xGoTU>m$`3V2CkHN0K1XL{X6a<4o@1+7dq_Guecqb?-TJuI zD1`kH%i+{ss2h7sxy}3@c{#E~e72p4u)%^GQ^Z)UmZ|wEm=jd3~*0)W=addEO(q%ac z=Hg&q8|XL*I!}YHGhppkp!*!?xd3`Eg7sg6zHh+B??C_eVABs^;1U?T0)~DB!&kw` zH8A=!82bf`-vAT8g6+4!NfC(@GCcp%k025#WOn?b6fzl^HM~ADBxC)7@kjB_5 zB;K-`x2!JtEvtDF@+9O*$O#ghAi*YN6S4`}gls}KA)Bz|O_;~gDN_U&^>a~wIqeo+ z2#Vl^Fn%tCB04%%9G_F@=%DFRVOl6mnlRmM=iL8%Zn1rxS%nEY8jKn1s67_ovp7U& z%ni}mQW`ABX-S-f5E;`o~UC?Q{TxOl3dpH%+!?91YbSpMF?n|v%dits12|8BW zB(>qtuml|(n-p2rfvyk0+7ChZN1*3p(7OSw{}l9X0vkUA{hxzPTfo2)x4DCkhop`b%Shc`@T6S4`}gls}KA)Am*$R=bHvI*IQY(h5S7;C~jf)1DDkDn#^ z6f6|Mzb2JC!3#lN2t^chDmgf3SVM!Pp)=r7F)G4N0V8wHx(u~dGmVS<{Vf$ez>9l6H6kXEkP3M<%T90j=GN`0e zS2tjaj)0`|w&C6>I(c_O=jjcbq|73ZP0B1kfZj`B{T0ynBiMKq^j`y;eg*@-fWaGJ z=vOd&3yk~*M&CLt8G9RyzXK-T1>2W^$)#Y&axk?5>{{>9l4($5? zbToiYAL#l7tZf9{8$pjB^lk?0n?PSP*cdo0B_02^!O|(9zQR2}ho=VaA(f~VWz?XAJM%Jkih5`hg!KT^Fh14R7`G3gIv=WHIz&be*n6x+`6NioOP{B}&*IUjW#EV`Yu_)2~|r|sP3 zCx^6i?7BuY2^x`N5jLaR`;4_h-!~Ea%H=+xihF&XNm9+}JSEi}9GFyCc7Xmb!KR&H zU^f`t1BUj3;eBA_02n<8#twt=BVghf*nS*Lo&Y;ef~nJB*BLPV71(nQ%v=Do7s1@u zVBa^OqYHF)gRV4Kdl_{1fSz8^n*r;80)1Jq@jB?wflW8TKpz;aiAgml#F_<~sQH)Fbru<5#RZ?GCC=C6NS$IBhoX*{Qie>9) z1?f4%uy1Cx2TWl+X?Dqt0ITg z7}T^ue_~XdXUJEK(j76$tx0Z;`U|FizhJuaf=MmYUoflZC+go}RxSE>m{m=Gp-e5& z{G5Q4b8M|AVCXnv7mIG_JZOGSO)9JoYUnh3%+`Ah9iz>xw*7yF8#*{JX|lWnhTa9k zOTfrdFuEL!tpMXI!Ne-C{XH=GKG?AqOsxaEJ^<4nf;}IBnUBHj1~B(2*tZFEYzCc8 zpsN|I4S?=#peG1=Tfq7d=-UN0hCzQT*c1T+`@vuo3>^Z)F)(rzjK*Q;#M)@+7-;A) z0Vco%m;e)C0!)AjFaajO1egF5U;<2l2{3`uCqP4oYmm4GiEEI?*cv3>qnh`qF8MvG zc@pv@l_MtX%n`-{2IxGDabsR6hW)6*w+urID$1Lw=RCZgGcp5o zBGSyUwS9O*>N($R|E1`9PWPq!19TcTFB*J+PU~9py_0dwwsN((#Zb7PgCmn3%SkYH z8jPO-6JLSt=fLCzu;U__`Wo!|226hk_IwX!egLzVz}ywE??=#)0i8dAt}Ix49dzeF z&rQ(V2iDgdk@Pvh#zmmt2{tVT11>PQ3=GwR;W{wl2BWLNm@PgeiORalxz~1NWhB??fU4QRX zy!R>V#J^2@^R;QuSHJvPb+uiYJ@1uy^IxB*K1k7N%^n_H(Wz-1py=T5PiVluU zsw^LY$&bN~4Pfe1uxk^T{tWE-9L#J1vtNL@tzh4F(9r@qL!fIHSQ`f2t)M3YdiR6% zQP6h?Y>a{aqhM1U477p41Q1Aks5wfup=>XWX4a@|= zY|Bw8>EuF3IY)NfC(@GCcp%kKvyx-I;kI z$P1x}l1>%86HT!S)Ny+2s^zWQo zY++|sL59u)J(>skHN)Ni3Va-E|6>-OQ_mT5dd`-i^qi5Ip_7qjj;+RZOzJt#8{wks zIW3pVtW$Igrvf_ABulqqlYOPuEIs|y@bnz3)N|ULro#0c9GTQuc7wS+VBcQQu^)6s zLDwO$HU_$nf}S|&Z3F8Qpzjpe*be&7f=x*oB|`?g3&H8)(ytfVB#{^ z-UB9k!Hx`=`U&jHg6ZpEPY%r71hai$uI89zp96Hf13KS5CiNWGl4I0!O271?#vn@j zkG}vDU;<2l2`~XBzyz286JP>NfC(@GCcp%kK*S7=7`NfrGZ^$9tNrrc(_vo)(T^smd*C-rT) zI!)gy>STRuQ6JN{6Xe&5oKa&?(+2&CQEkR|8f`|^B)2BHHR>;z{{4dK&I=~BOn&~+TF zJpsB;f}YdxbG&Eh=NRbcFaajO1egF5U;<2l2`~XBzyz286JP>NfC(^x(kDPahkg$I z9Qrx*b9k$CHX)mkO~@u>6S4`}gls}KA)Am*$R=bHjbM*sP#R(fQK-ZqD6%=8lY~Qz1+RxH*#+icx)3;o~?vXUw@dn@i*7 zxTTh3YiNv1H|J1WPto0+#aHsXIc?`IKRKkEW7mi7Ov2C|#UiAe)9^p61NIr!&gZR< z>F@g!-eOLml zA52gwmad}>)6bbTD(1{9_a6JKq1>c}!uarySMWndRYQ^=CruT0_(d)_Pa=D$8qebDTjnp9XF)X!=5nD2LS zn*Zm$`!Kag{vP@L92}YSST=%wKiISx3^akkW-t^0!`r|}5RA5fu@D&F1t!8^dn=fX zfF1k6R21ww1g2wP&rvWF2eWNpE&=wP0v%^S=U1TX99Vk+bYBELUxVIn!20h%-}hkS z51{`N*mMO9{0IiG!p{j^qo2dsIZS{FFaajO1egF5U;<2l2`~XBzyz286JP>NU<@Qc zKZkw}{T%u^^mBNhbT%QIkWI)YWD~Lp*@SFDHX)mkO~@u>6OOSa%;V>9MgI6%kx#!u z5&Ub?xHIxXkQYJ`{T$`L*tZ$3Yg8fqrG7c*{wI|qrt8cRh63j2JkS)X95HjIvKY$E zaHX9pDmqrhVz8Oj^An}^W0X%Ua%+-XO@E$PYwmxKe6O|4Zjw5UwNf{0CS&4^%@UAyj;%FvTq-)QjrB!WbhtHG=X%&h_YyrAO)(D@;|KIj!Q=;w)Z$^=x~M(6JP>NfC(@GCcp%k025#WOn?b6 z0Vco%m;e(feFAiJxC)7@khltIjIBcAt*Uve>XP58nkOMoLY{=29Kp#EY(h36n~+V& zCS((`2}|CDc^n;+BDkoZi~7y9TX-QTf)~R0xe$uz=*;BYoPuWt&6G2P77D}dhn&lG zzov8dp1DO;cI3wupy?Deq(&b@V_qnV@R)@kQqYGM@oj%lM`yCsValZzGpb{B{wY7y zXe(FCf3U4n!Z)k(NuH-Lygh7JwB({icUh>IkwIYC^|=fJ63c>r}Y72bb4>DQ>}m1 z=lQ5|p3l>dS7g5K?5{SMIgCD^zV^zR0n z_JD!CU~nH8Isk?bf|0{u^avO`2F8zri4$P^NicaD>^K9az5=_>LD5NHprT`-qQeB3 z025#WOn?b60Vco%m;e)C0!)AjFaajO1WKO(6&)@@;vyt2LKka)Lh-mSXicdO<} z$diyKA%{nBcm$h}O~@u>6S4`}glxi+H(?${r%VxC)6X^i<&;}^A;>8llv~Elg-}FA zrwZ1ICR$cYG_05CRy5J}sN6F}?wG2H#WOToJi5U&T^dYs8jOayG5JNE+xzDhTi2OY zke~A~n$4Z>Rp9Da3V+PRo2lrGIYnp7P>RkQ(i<{JPvy3%GeKQoH(G@L-z@hV?O>Ts zrGKxv|2^`()-tM+(y-He$mpW1Dfl(chfy_-)+Md$NN-xXcaX}MfxouYqU1wL<* zrCYJd%2I2Vo_=b0ijKu*Rh#3(QFL%$8tl6aI(`J5S3%b` zu=Z!r{R`;10eXK0>u-U+-@wMVPDuLS2Akdi1Mh;tC17YN7+wxWR)Eo!U~Cl_e-BK& z54Nublk32a55UxiVAn@r`eU$X1DN>~%x*d%6`kB?Cpbl?^vf?wev4NA0!)AjFaajO z1egF5U;<2l2`~XBzyz286JP=*Pk@RJ6&)%%RCK85@Q&$hLN+0rkWI)YWD~Lp*@SFD zHX)mkO~@u3V@;Sx(cz+e+9}ku1nm^L@_yXdDg0~7zh)5?ok~v58P?RG`<0Z6QBigZ zn4vQax26yaXAb-u>mZN0c$I?Am=koGhZ1x~W`<5ix;eI*Um@t!p1DY1S6-x=ov8f6&ODUCN6;O7s2G$V8=IL>N~LO zdocY2*mDWYTmiE`g1M_;-!;&Y1)bL+=(utebT~tY2`~XBzyz286JP>NfC(@GCcp%k z025#WOn?cDfdnY%a0wEZAaM!O7+ZqGTUGN`)g`}GHBUmGgggm3If9cT*o16CHX)mk zO~@u>6PCOQ^9VX-MbOWnpJAk*!3#kVyb#9Eg-}F6heLA;8yHk6I$s)GN0~b^qE3Yn z1;{yri8O;aHU+*k9V*VG0gnpop`SD6{G820`8gvqK*u4~99!Mtg!FTMjJ)@MS|8V^ z-8>?v-8?iQqz4_X-=c`|St-9K-%%1nk zy!o%sQx{*!@8`6gyZq$v{2U?uoSMjeevVmf_A%RIFrvp|nIzxXdCDX`7I|pW zW4Q&oegkXYN=Ulj20ib9-gm+JC7^F9*ti_@uK=4?f`L_F@I5f}J{VpLM%IDR55U-m zVEiL6@iExG0Ze`hc5DJupMhPUgXt|`&lg~3E12C5=5~O6UxJQZpfe1*TEW@~=-v-{ zq6z8ecn>8wK!*c#m;e)C0!)AjFaajO1egF5U;<2l2`~XBzyz2;=@X!zLqCUp4*eYZ zIlNgqn~+V&CS((`3E6~fLN+0rkWI)YWD~Lp$5<2Q@pHH$fBdY-r(dB6{xxabnRp?{ z3!#X9&UERWXyQ<{CYoQ>#F_<~sQH(4b`}{l(Y8#YWwk`ZdWmjD?s`=2nId;g)x_c% zn&^E-6K9{*MC)_ff59N0DtYV+h3J3G!t=olm0~G7T0xG^K!Ij}PgBG%7eLJs^0ylN z@QeHE0z_@rgt>?ZQs9rO5|gauQp6FeqHhiAU-Yd}eL~-w?rhEK4E^ge^+|nOu1?dp ziaJ@}TGYq&?F9L?VlmhZYTBSbF{;h@PNU7Jn&j3bw?_R1)4yLZ-Fd;Jmgz5;)$;6hWAfOL8qN zfC(@GCcp%k025#WOn?b6fs!XcKZh%jxB`hQkjB^wB;KZ)x2Z1qZK`<^@+9O*$f*&W z8o?%H6S4`}gls}KA)Bz|O_;~eDOUs+^m9SKfo2OY1V!*d7(W+65&azHzu31Ku4_~w z{iS|6=l&;^L-{JdTi2N*Oa;u)`Hwhs|L-;ud#Z4}(v}(3N7WzG&v{HbH03tS9ZzRm z`Z+$S=GdCQgP-$-_dwD8obF5c2k10xUNrasonD{WYS;aoN$_(N>E{^L-oM}L;tVe6 z;LxPVvKI8O1Dieo10RCHkHFB!V0Z%<`4o(90%M78KDZZNY4%>dKtYEIFaajO1egF5U;<2l2`~XBzyz286JP>NfC-d70SY=4bSUUh(4nBi zo29b}*@SFDHX)mkO~@u>6S4`}gls}KA)9cFHDMk>hfDIu&ysu!7K-3slgeEgF9dlZ z6j9Kb$)Py~4-A?q2L>$^rrQsBA`Wi7%q^;~BR{TiM@LSXD(>)_L4MjyeXb2PopRgI zV{~qh&lzP+=b!qVopL#N2a|cq)j!Lv!KSEh=v#}rK;KSK7uby!q5n6_*Nt|tOsCSn z*WCXe`Ce<8-6WM4Yo%_#Ooshpvjn7}V{6@dN@_Z8=Y>(zbXv2A2iJ7k)|>BDa*T3j zjvNto@6N$J9UPnVS-uBDKY-y&VB`uI{Sk~^1>@Ji#Lr;+FJSTp*zqfvx&?Oq2BzOS zCE4>fn0W`xz6<7-fPG6rM=j{A16^*gb~WhsfSxs=*9+FK2YvNmV*}{-flZ%)fkrU6 z5e)gk@MbX51V)=rNlzyhIK@FarC)?rwxDtsgk+>3RjIBiC{i=Dt>XP5DnkOMoLY{;iAHnevY(h36n~+V&CS((` z2}|CDc|0AXBDk=h3;P8P7hVX8;Ds=LE`%a_Iu+7A(ZtzjHPQNn@H_x;XtA(QNR_hC(~nurf1f}9A9iK-yjSMUe|?@>S2w`XG0RArPYlb^!Ldn~Wjh$# z0mi=s6Fb57-C%ML*s&K(?E||Gfa!x^&tWig1k4@-bH~BH6QJW1=xhgFXTjPe=spj6 zIzVqHSf2uY--3-@puZbzN`ryRV6X=a^@8CH82Jf|X2IBXFrEVwH^KHkI6BFi({yy` z=r92$zyz286JP>NfC(@GCcp%k025#WOn?b6fzl^HM~99M9UVG4baZ&%bT%QIkWI)Y zWD~Lp*@SFDHX)mkO~@u>6OOSa%;V^ARsQ%{l~2b)&WE6EMA?WJg1iul=;%~IJJCeT zYKeyR65Wa>+8&jArpO&rHL-YxMvX^Tn5IjGX`zr2<=1rjm7Tf8_H||zB(R0FO;cUhH`X9W{yrm>N&Q~ z<8X9NU0C*iSs&NEnMdHwOtAj{?41vM6K8(rNB$?Hu*r*)ER(Ri(6p4c+iu#lo0k5` zwN;vJuO+=+n{w?n*QU2zdQFq;rI+q*_qGuN95BQv0vs^JC<-`Wh!G+fV~7z7IKl)+ zC}6;V5rPmDf(hYwxw)_Q0ecJ~!#%8BabpW5!U`rty5SZM^cbaagx!Y-zVuP-MxD z`3LVc*3m&%%2ichWE~y6H|ny?0z2k_oiBp1d0^Lku;(SPcOjT~8SGmG_AdbkUICre zVEGELq6Vz=fNn2XwF<1R1#8xT-a4>$Jy=%{*84!eA8dFFY-#|T8^K@`*s>jLZ3f!{ zU?d2(?*cnoz|IgDYX!UBJ4hX!o;L32Sn@yrBJUSzWdZB}JHQUG1MC1hzz(nj>;OB! z4zL6406V}AumgE_fIB)o3yEhT@hqfKHVcU_SIw8J&im!6c_ZYFkT*hJJ%U${;3VWE z9ii}+*%5Ns(V4(&=NMWyh)Oz| z+qj$3*~wxR9b@gA=h3ezHFLA$$GM?HtK$^PUsYDgDUzos_r>x(%6*AENx8SmH**yo z-}M?gH)a)`7?pEG{MB}9=xpAzD(4NIcu(erPGH}r@wyG2z%tD$I;FxU$2$x)bnxD& z$+92p`2g%a2qq4JeMi9lqu{`C&>02GV_?N;u(AtucY{?uVD)*hrWf?a!P*2^_X$|v z2l_9A4gFx#6|i{#3_9A0El#j?HrQ4UM&^R;6=25#u(J}3xxg+r*s~bytpXEE!M^Ht zYUuQ@Xy;XQxS_)iumkJ>JHQUG1MC1hzz(nj>;OB!4zL6406V}A(3 z;f4-hF`bi;laP~;laP~;laP~;laP~;laP~;laP~elqF$?h7Qlkzjfy1bHll8HPsx=w@=)drp`F2Ad7a6`KpcW1&0s?SYzl(SyTD)z*b)L; zTfw&Xz(^a|9tJxiVCMla)(&1Z!0jBKf5h{Tc>d8Sn}5WYspiX6 z=lwF(ybJ`FT2jG51<{N3cq~ zBiuSWLJr$Glc{xLmqIh_lJ9A|wDftqthk6^0lRO#>mf?6_iTK|3&n8K^a)A(3*O*DD@h8 zt*yAMuypPe+XP#IB$eGwUzUkdC-rlLtLYGxbS`-g+dK0lGf{tNOs|%V14E=)D5g4uExz zL&SO~=${QXl!Hxk!R87uxBzUa1Y2ETn;VQQ2HUH^j-_B{H5gj~cGZAA9;OB!4zL6406V}AumkKs-W}kQ4wrPeq{Ag0F6r>)(m4q^2{{Ql2{{Ql2{{Ql z2{{Ql2{{Ql2{{QzSrTR_>F~7tTW4B6mn^Ij|7&vXu7Gz0+7WVC(y{&&mu`#}j>b`m zDb>(P|6;-o>+DRIq%4-vxqheSx}}?pFL(k?94Y-qs%~6E=LvLwO44uB@!*Ymrz8Ez zQ>C#~YAF$tY*p?p^4FF70{N@Tec|AJkvv8DzF59Txi66?Dfd?SZsopo@O}*4Td4+P zk$Wu47X`8pKUp9OA}lI_Y9#qxC< zI@g*xxo+sVshlG`a}QHP=kS8_IdAB6pUb?Aj%WRB{bh6E#SX!DY=-{nUk!3qr-3-F8;7>j{jr@@{su(umb^niWm!Tw%wAPzb&g5{rr z6_>!u&p`L*VAWNy`U|jT=3%0D7FavyFcoy_UOdbN9WLmw1MC1hzz(nj>;OB!4zL64 z06V}AumkJ>JHQUG1NnD=3pzXniKig(6r@o$1&J?K%@?cA`^BnxBjk;cH$q-Jg4d4V zB;+LIB;+LIB;+LIB+PpfW+>q)j;nvv^a#+y0pI6Swdexx& zY1N=qPlYliy`!UkOhEn$FhCv;NR_m(qB-ofrCDYHzIU5{!WeYngr9gj{{UON%5z1Mcz1CxIE+@yzRJo(&=8D-YTo_Xq-XTSf9 zJW@?3(6>jwrW4+*sp+5$MSth8Mvi_>2XBq)EEQng0OTm_E zuyqC4Rs%*nV7nLWSOs?0g0VGVR~^{19_+0L6F#ud5B9$W4m5zytzh{!u;Oj7atG+% z30A!WR__LD-UYpTz}mfF-9E7XebB!jZ1?~*ou-3a)3M}#3P#>9(#itZ0d{~LU7TytPN62AKXB<^e?9$-}?NZ<& zyR_yJyX1P*E?s`iE=3==OQ9Kd$@jFKt321LJXR^Zz>=}O96HI*wmS+AsXGb=j&0;P&mqEvATsG6g#=-ilZ(urSLF5AALRD5JqeD1UredpO{zdO|JiF=EN*KrC&k?dNgSv5z$ql5QGU6#XO zQwP|53=DRHEm5#F2DY6BBVAy7H`vhwcAf`gy`8#VpMZ%zuw zbXs3O!b|CJM~59?2iO62fE{25*a3Ec9bgC80d{~LUcGf)u)QAa@PVCvF!mPM)d2Q1g1t>(VmsK^4E6`WfgtGI4VJ$P zR_p;Q_k!+yVAcCz^?tDC1JHXAtUUzQ9Rcf)g8t)R!wInIB-nfk41NfJHQUG1MC1hzz(nj>;OB!4zL6406V}Ai~=m3OYO`|JIq3&jkys z#Q&OHxs!NDpdBHH1)T}JaE_rxgQ%gSxsAIig`F&x(9w2t#@+Y^&E%Wj(4j&QmvoA4 z)=~>C1as^*S}X@E!j#f|A>G@RCyHg?^-4NFP*&L~m9MR)Ge&;Cte{j<{x70Gxu6Uz zR%lHBM_7mvjRAHjURU z>9|~)Rdh;3sT}qe3}0GDe-$0PH!8Dy0(SI)otME_KiG8z>=^)i9Ua7k6YQG}_LqYL zb3x~Pu>2*kVj)=hGU#3eRxJUmUjb{Df!^g{?W+A?QEa~vF zIoT~4G(n0!u4(6JzJ21xly$~QmMjW7daKzyfhJ99Zg11^2kQN$H{w}t=Ulg)v;KPR zoNKL|T(@&#RLl|a%A?fIdHao1IdA98y^y(`6Fz$W9%JpCQel&QXAHG-@ZPA!5(Im8 zfxRtYA_Vreg8lD-18tylKUn?&SaA@nJOsLrfK^Aq>f>O|3DA2ItUU$ReF)Z{0sUve zhI3%k1+e)eF!(XpauIC(6l}W$Mm_`EKLJHQUG1MC1hzz(nj>;OB!4zL6HcYxbD+|J>44!3i- zox_()=OpAL{4ikUGhC`mzF+nmmEK)#?EYuT?#KCG%hB1RuH=XqN&H*?$U_;+abC{RPRAx=q8^+7)%wkL0SbW6!FgW1xU@Kgu;W z^EWI05H9JImWV?6tF$Uj2`ytYMY%7Q?@{hc%esg+`Xp!F!`Jiwo>`g9D2}=PO|OGO%JfSotdGUI|vc23Efg)~p7- zZ-BLH!Mb%|{hOeF1K6+;Y}y1iZw7-~z?Q9G>o&0MZ7{L}Y~KlXyaRUb24nAnU389VqEk9OIIXCI1sJ@_v0*7Qha$1MC1hzz(nj>;OB! z4zL6406V}AumkJ>JCJt=xTM3=ka!vrPeU4I(~$T|)qJJuykDuBH$vVBc_ZY7BY5Em zPC`yXPC`yXPC`yXPQtt=VTO`UiB;lR{XDDR!u1y35v&sL2)E9Tki(LW^{2RWW3+HI zj!I0ahEDny6K+^%XS!67#WFf(N;hL)kPP3Z;zzil^OM)mxiQP=IH{Z?TZZDlX<-RdC%Oho$X|n54&7%*OS*GLB$;xXd;;q7r@$&z`BpY`ir3dQ?TI@ z*z_6L{5crB3buR!w$408Y?}o}=78-lf*td~&iP>MC9rED*z+>jy9i7y0sCG7`4BhZ{QF(BVs_a}shAauRY9auRY9auRY9auRY9auRY9auSZR zB+Ssz;W_!Y&YXO1SXd?g*W}J!G4BYpBjm85b3ZSgll8Je_tUaLFGvN~Zq$2BFX^Zs zc_OEU9s29CDCrn(*<>u{OhffX%I5YIUY47Z&NTcvS*-;06ke8_cbfg5Lv&kt>;PN@6bQzr=k;np3LJRL;Wu6lGYjkfBR{6%1bllX>5uVRb(uwxe-L#TU zpl^?UNykxJWGhqpIAhVrv6f1-NKv$Gct@uc9i7IJIy!i3)Ma@Stlj|DYy`cVz}n4V z-4?KZE9l<_HoOfs?Ess1g28vdmfc|MyI|WMFtQhH-v@TQ4|eVcV;_KB2f?00VDAwy zaTM%34)&h_2Tp>{(_ncQSkVnu_JHp5U{xZ-l&d1g{;zNytgaNytgaNytgaNtpK}%+S#(uu43spC|Q8+-~6=!7A~NaO>;{ zIqc|+quz;KI{ctr3Or<&);waDT#wqN%a7Tm=;L-NG{Y|Wp0;z3=UR`)DuowVQY{_z z7qrfex}TH&waLcL1WC%GqLWpxCY$b!`Ze@dtKX^S>j`+{SygLt__C=#nJt~WlvYov ze7ABxhVHFYn6bz`7Uhcq**ArrDWLvHA>9|!eSz{sVe*N>!6yo-bAul(l8@b{{6vvl zs{BNeY*(Hrmi-0vfQ4E)KTuwf-qOM27OJIOYwL`WpD!yYm6ZRB=<$Lwuvnor^=MJ* zHS}6raakcXUThPT^2^;Qzlc&V6?BB}-Q(2K+1Vb&(mNmW5T~e@*V)74eQhJ3%ZmV4i-$9Rzrw1n5rMXk=T;x$oOp~d^MD-QBZfk`~EKlURu`?~Jiq7B)KbjUz z4!?a0@2ejz-lK`?R%Y(D~a90fa%gRv7}*GaJF6xjPAm^cIWodx^PfddynXB;d~fEAyB zm3^T5GFa6QR$l>Y20*W)lUVBn>t=)X<)D8q*iZpBEdZM;OB!4zL6406V}AumkJ>JHQUG1MC1hzz*c!0WRq96eOO4#8Z$)*%TzcS~Xv- zI`3Dj=8cdyLf!~@@d#c#f|HPwkdu&;kdu&;kdrX)NtmIaW3fs+qn~H=7jm_QcLb}% zJHoBABjm84Gl7@RF|=%udV!g1Uyr*fg`F%G(HZK_q)Ie5^oyz7(2>7Ni_v7$(7Cp9 z&dpgt$CsamPLRquB76!BomusXoHumhJ((Lifqk3C>o#=a-lE}UoDy1G$VW>T4Xf>x z(uzWX#)9Dwjum6&=z&6Nve0{@Cd(^e>oTxyIT(2rY+ng>yasl@4#rl4U2lLrYr)=i zVB$@%Zv)uB5ggbAIvc?9MzEp@tlSQ|o5898SRDjwc7fg&ur>tNwSx8Uf&MnIAq+M} zz~%#BupMkU47PTFZO6b!C)gfEL#HFg4INAVXJ6#~`m8K~9bgC80d{~LU70a|gq(z&gq(z&gq(z&gq(z&gq(z& zgq(zWq^LvuNj(ydd4w+f@8@ynfE8Yv-)bPdg_<#T*gcc!JtFrs6z}}C+#6__0Q?UOM zIPe+hyaJXFfEA7t#7ZaVo()!&gVl4vnhMan0IaP9>s(;H8}u&*8>+yjrC@V47+e9i z)PSuXu+0lbR)OucV8;OB!4zL6406V}A zumkJ>JHQUG1MC1hkbeibox}5wc>WR3KN@B8kN7gxe3|OJU#6NjLf!~@Bji;hc-06_ zLQXdcY>QnAFCa87CU{m5x~=rk9rVa}dSIemnmffV#lK~j4nJs@0uR}L z3fAG7O7HyzsW5e$hOZ@^TdoQ_c7qL_X*Z=i^VChR=TISND*n>3H~BVI8mp|6Bgt0f z-Xecpxi8@Lb0n$kZYtK4iP9LAb42_LG<0^)@#egtGt&Awj##s-+tBe(7)yX%pMX7mVDDux z(GT`rK|`m1fEzlzehxdp4zL6406V}AumkJ>JHQUG1MC1hzz(nj>;OA33LM~u4mWhT zp~DRwZs_o}(m4q^2{{Ql2{{Ql2{{Ql2{{Ql2{{Ql2{{QzSrTSw=uW&22W;?yO{>7>S}?cJHQUG1MC1hzz(nj>_FZf;F1ncL*i*jJPm1-O+(_VRrA%V^M18z-UxXk zS1AnT9`yOFE?` zqHu6Uog#S(^$X}y-i?9xsb8=nMaZ*1=xK^B`lFp0A_usUV zPM~j(eo4o(NK?|0sHEdKY0;K+Y_jX5W_g{l^vdXgLMpS+d!sT-7_5kZl?Om~J6LrX ztnL77j)C4zur>;OB!4zL6406V}AumkJ>JHQU)-vKV^a7l+tI$YA>k`7-ros*E0kdu&; zkdu&;kdu&;kdu&;kdu&;kdttfC1Hk=4o}O!b*ANW$-*k}zb4o2EW9Joj*!EW&NwQb z*rme{+NHolc4^HccFFaqUAp|3U5Y+#mqIh_lJ99d7kRD~d8|@+!C*%xRo1!o#dPV$ zR(2*x1zB`-zD)CGDtb1OiQAXy=qQWfWYs7dVf~*k^G?k)lgK5ix0DD;wkr1)`RmGk z!C*IMbaiz6*IP*E+5$Sc?&!Fwo+CUx=;(BZ&gQ(M6BuD3o%ogIvh5qnias(e?sMOr zwxaJm`|NkitdBkO)HBb1{~0;3tjJcTJU^D6FSC|P)PV9#9A43}37Z@)*DS7+s_5XY zQI(|ztoDF4UeLP=tgQv>)`0bOpnpBsP!Bfwz-B)fd<$%809zZuwk9yL9c*s~I|5*5 z5RB~tyIR1W5ZK!aCf)=4+Q9xWI1mAy2f^|~V8s!z@+jy&4pyB2t51S8r$FzAVC@-H zbn4D>MaPo==@)swNGl6q2iO62fE{25*a3Ec9bgC80d{~LUsloJuRA841H2oaSw@>7{t}`vGevX-%&EU_{dSmnkv%lSs7n^C?e3_q}n!9Cy zvM=qJujt&RRC;dCGCCWsSJC+aiLr=WS9HA8&Jn)G7*%xkyfyQWYC2k$jj(J2r`vg5l(Dmr*;RAo62*7bt*anPRt8$JP>`oQMPV6Y!-xdOHh zfNhQ#G2#T;pm!Zu`zBbo0j%E$`Zs|Mn`2bbY1$IwRdl$b!w#?m>;OB!4zL6406V}A zumkJ>JHQUG1MC1hzz*c!0j}t9MTaXoT+!i*4qq{ylaP~;laP~;laP~;laP~;laP~; zlaP~;lW>$JVTOthPs-?j&u#wijda&(Pl_=Eb3=-|Ck zlcfo4+7321gTVmU5(Ha!fo&~dBm}m%f*tRHoo!$&40c7po&#WSJD4~O_H}^$$H0M3 z(0LLpKLu8N2v(i}-Dkn7b71uau;wGs`!QI15v=f&0m1QnWu;? zvrbV%r*+OLZs_EH_C?;W(8>bX0d{~LU1Rr5y38zFCmymSOF9l=S+NytgaNytgaNytf<_aw~F z&?&M?x757!TV@Rm?u>9}g!fHav4eMnFSjG)u%W}N=47{G&;%*^czTOP{m2tHrl~Vd zlCo&$=rw6{N;X+OH5px;63uNK9^rycX^AM5zpAW|QzTDO?u+Gnl=~8Sl5%gA?^f<_ z&I&r~uUF8yF)Qc;OB!4zL6406V}AumkJ>JHQUG1MC1hzz*c!0WRopL5B-ET+rcy z4qq;vlaP~;laP~;laP~;laP~;laP~;laP~;lW>$JVTOVZ&wnrYa`WH0I$)L9I_|d> z@{TY%cZ3`kbS6{r#4d$q*d^c7c4_JJcFFN$D(%d+*ro6SLgQkBX9c0#N)O#ZkBp}W zCfcRBQ|wawTXyO2gLWzKko~7%DW0kH-d~WU)NLBR)?2tHSDhVuRwW&6gGN)R$>F!p zqnlG|mrA*zqcm?uS3@UHOXx(XoFk&%)6~#8e<_snhR#S!=s04{vTj4ivtF}=PAM8X zjRi)Q(7}77CQCcmaTx6E0At6%u1>Hg3iigp#A&dv3+(R(2YNu~1+e@hu;OE|@*?Q| z6s)=gR(}T8d=7fAg0)|Obu&*B>t})fIbg$!VADLXc|I6?32a#iw!RFuEdnD;!1h-mlNf0@wj|fE{25*a3Ec9bgC80d{~LU6IY^^y4iaCfny*xy_bXNNM#vi>Z-l&X1TP%HNytgaNytga zNytgaNtpK}%+Sy&;WcNh61NO2qqAkeQ5vIij)=$6(Ajq?n)8NE_qoh#=y=x8 z)?Y&>?oDgx*kqrh&`3iE?~R%)tH7RGuy+lZr~~`fgZ=g3fDd$T0?Rjp6uE_VISD^KG?h;41NH%90Xerfo(^?$WgHUIM{Il z>^up^PJvw?f<0%z-m_rh9N2dO4W0gvxS_*q=&%Fq06V}AumkJ>JHQUG1MC1hzz(nj z>;OB!4zL5GzyWUPa6^Y1I^59Vh7MmZos*E0kdu&;kdu&;kdu&;kdu&;kdu&;kdttf zC1Hk!4wqB7oWd=wVs2?!CEgKksU0DQ4W0XW-JGn~47#7z40=H-ymq5lNiXTBA9*6D zg&q3qvM=daZ(?`me#yow>5S%;bbdtQ3Pg!qO!tL!|24X|2&?=9<-SyYUb!D5KVMc* zDk=XL(enjmV6j4L>d~UqYv{GM;*)mg_UQL?!YedA9h(>{yLKDu>EOLlpCtkIe*zBlfzHps^3TDFt6=39pnK+r#Hv|f z^&GI~MbJABtep?my#&@T1pP0A4U530C1CR_U~n1OvK(xE6>M7xMqUHkUk5u@gPm`H zv9(~=IF`QA>;OB!4zL6406V}A zumkJ>JHQUG1MC1hzz(nj>_Gk<;GPc8MBO5jHB*} zT{`@rT?#y8m)1ODmt2q9rOS`mrRd{!DKx__`JT3Om*-lS$0~&vSW-0&IHMlMNKC|V`e&PG-Gej?FszJ8UE56`ryMSKflC8?UMgF>S zUoa}y((xB4-J24ni<7CNbFG_`>yD0->N&#oQWte}7H)GrV!KNz@Z3%Xp8LkwERU2~ zrpc~HHIF`AW|@vhCo8X=h}Yh0yX}EVKYVV|!!w?IZpMr<>toM6^~|&1e?|_Bu$E5z z%5vHE4dv%${QRA^qVGKW?01KL(B;x}bR_ENII1k#wRFbH(QTUI&RBY7^gtnXS?Ik{ zmt`kd@eWwI8+5-5R_y_+_kuP1K=1ot?S8QC1F-%e=syHD908k-g3ZUl;0dthB-nZi zZ2J(5oB`X#7Ae2~u>f{}9bgC80d{~LU>@_%3RY)Rd$}pbzf&%Rt25w7jY(|QX}Q?HV%K<^*cI~bKR;Lcf(poBXxA> z@t>@k&W7vNbdpQyT)WTJQaWyG=m^hI)O6ytYjR%GX*^$S8?Ur+#$oB6v8CNsL8&D> z<{!NGx;33LgefOX5j z`sJYiRj^?t*z_9M{5lw14Ys@iwyp)+)`5{X!S)Sc$40Po6ByeJc5MNBwt~Iez{J~N z-wv>UCphp9=xhPYLtsTKSot34ZUd{rV08qnIRJXw!P>)U>C|;_OUJ@39d>{nUQ1*`dj)p@^QHE)Ex z5%NaJ>qqeV5uAjagq(z&gq(z&gq(zVPr?i>9Scu_|8kSyx!c3to-fgz;LV0No13%Q zI_c#X361`Zv#`VN+n{X2*|TzZ;a>*HBj`w|`(Zf13G% z&Rxp7JEij7%KaF+x0YIjMeeaEUlho`DfCPMRYwZxzL4$lJiv%sM(=>gEXF+A~zp+3ybIyr2{B$z0G0?AtV6x1bYP zR%9Di&?yi_a^QOzyEqviq#_HwH7c?k2kTCN^(R69DX`%~u;~ogd=?Cz16wYDtsjAH zAA^yLVEdJHQUG1MC1hzz(nj>_FZf;DQbpbhx0y1syKv@FmkZ2{{Ql2{{Ql2{{Ql2{{Ql z2{{Ql2{{Ql2}fBHW+>=N#J*>N%yQMWRIZY_?=9;$(b~S}gS5sKv4lY_{;XK&X3pU5WU;=FU1nr#GK5pmm z@;U4PJHQUG1MC1hzz(nj>;OB!4zL6406V}AumkMCC~$z=IXwS}=O6L>qfs{hh_6!3 zSE{56Ep>Z+6vx3lVrHAgIN5<0w z6YbL6DRwFTExUC1LAw-q$o^BX49`@0@9OntreXaU?P@yeM{-rzp@nL)Xy_Oy*Ic)W zGY;*WlIz|k<278;DJ>C&@>gjUn-W@?1?%9Hh+_F3<-SCoq}*EvSI!wFE9m@yL|H_s zd~GG2G4k_e1*MYme-S-iPzDw&w5A>{O1*|&Yb!1*q{54Bg3^7t8{HRC8l--X2=DHu zlFp)@k8)np8EFL_N32=aFX{M;hF5P&=&d{Y4V858-l)v-8QA(c*mf0+d;zx4>?U^1 z0z2n`u@}Lvd0@|cu=gb}u@LNg8SGyK4lDtkOTqGLuwn&RSp&K~V3ilFUIo_Fg5EV? zZ5>#*9;~ki{XVe44>r98HaCF5MzEy`Y~2pFHG`1=*dFYrl1|62ZeBu%m(XDc*a3Ec z9bgC80d{~LU-?81L)OT*D`p$i0Y?epLEYoDyqnbw_F0)L>qmz}_PQ+{PwcYl> zq#r&v>ERhqJ~v}Vnf0+}o_gll?>{4VpUb?Aj%WRB{bh6~12J7dLI zIeMTlq4!2jmUqF9Jz(cvFt!iudLQiB5B7clCJusqhrs?L;J{JP*$I|M!HO7I zc^Y(gfmPjLbq`o`9`yEtwQ;a60oH#4`uo6!%V1MK*n9;H4uCC=v&2>>*ftxCl!NVa z!Hx>Ba{(Bu1iM^ksiD*3KFbZA{Lj8{Sfu>^#{$>^c7PpV2iO62fE{25*a3Ec9bgC8 z0d{~LUahQ!-mfNymU_1%Ld&~%Lct5 zN!M<4aMd()G~Ygv)4~q@by<{jGW2NDn>FSixgXse+qIh&UvN)n^!9XqL@VYLh!VM& z?hEPuYsv~bR{6%Pq!Xixj)Z2YU{Ky&YiU7}(c| zo=$(1dpZ{G>97Or06V}AumkJ>JHQUG1MC1hzz(nj>;OB!4zL6HcYu33+|%Kn4)=7p zr^8oG=OpAL%(wax?lIu~sbonv66n)$-g=W|#-_v&P@?7ikSf%g+ zORA}({({!J(bRM_-*>mlBX!&7t8l3 z_a*Wq<=!gat=yLm-jAVsYpF$8ApaDqA>YH;ouX6)VRTq z7RkqMQ+}dIE>(V_NVY3a6wCesdcZ=R9CUFqb#!jbS~>};=Lm`N-+ZTf!Rbb~*FjftAtpIy!z+MlS@Pd7-!2VirU=8S82bRCtLmi!p4L!V;PX6a# zeZ-l%N@E#!VDdqVs1`wbAp=_H?29r ze-SU1N2xGPrV7&wQlVWc zRJU`|zkVXueVu996?E)3w=whlOHjxiu{<)3&ztdiGd^z?9X`!w__QNj45~n~ zJUs0$ADjNub{->#A8;moi3<;;kLkzJQkKb)Mf%c_<1&p(M|v>Jq2$jiuN&%m-=}*To6^=RsQWyDC|PsH>0jxbh?_Y~X4;ZGb}+HF;aiGh8a#@j|N zaNmX(my}90!k26#e^WoGcJR-3KdSuMq01+KHlJ_kvb$_~`k2bo7peUMuM> zkfRUl2=)yMF2n1;QtDk{5lVyp$iCk(r#RhVN32;ML9wr@$o3T|D5ZbkA~`V6lD%Xa zbc;Y^fgEbu`j_S;cT>ZE_(tJbKYJvTU9ObmBHG|RZ3ZL<8w&(2(kX|QuYM}ml9eSx zGjGS9MJdT8v`;z~73jFUad5-96aOr?mqyA9Mf3nkzT%o?F{c?9QW$QtatVdZngh#H znoDVE)94`snqvnF1zmhW4jsQzZBldNd97y0{Db$7ur;eR+vtO2U`3&h&;H>)C%i?% z@iA#`?xtsMUdB0gqu?1VRXjVYEJ@q55bZx&A_8lzawzzUyTIHQoi17@?ZPWkq7l&14kMz2Ypo)DiA29X ztInioLvg1%svx2RD+>%b9l2x`(cPtTX!b`tO^P;@cPi1yXwPXQqGP*DMSRg1IrL%2 z5tE|R_d}gQohGCmYjwjzvm+)JjS-F*iEe56*raGffu|C!q;#{&=_1+n;^g>WX&zY{D}@qUYAUW>R!|dgu)A zREHLfNPT~$D|*xLgSdCVCVcPRCWp?>+nRIHdXYz~!4Kwi(p$~9!;Xj=)&BQx6M^cn za;Wvwuu0X1GEa4MQAW~dEZR4rr#e_YR)o7r^}3VYCRG~>J(X%@FulIOi0Vl9SP^ab ziX2+7&ry`?eNiX%G*KOm8&Vx>_=D>81Nv$G+8x645s7wPIgxYGI>G0tO^HTY$NxcBv?lkwAKf8*Zwoo}@tI2|MH|XJ z)kiMmUU+qC`^aduMz#NKAp-Nq$)U&x3yaMqh@sq5sZPoqJjNuy)lKX9<3#ufsou2X z4U?+V_eGuDQ>Aq&#?_6f+{;FF|*sXE>3>~arj?FgpQIvcMOpOy*7t~=#WyYnNHstx6y>KLP>^%~XQSStHgXwExn z$JV><6fXB&a%f9$d5O8bF7WOXYZR}Y`l)NepN)5 zj+a9#<_(xso$j?x?%~Yd;3i>%>he_XWurQ_bi9ank?O0TEV1S~L3BRos*W+r*t$($ zwI)ImUE_sgvn+=?qi^P1wNCEgI4)f$tT)TTHR~QZwBx{ zJse_EqUi+nx09lin~QE(&$>r=_LFE&<8hOs4dtFnv@*2*p{{66?0NU!BYZ3Gl|wUE zeQHv4`mU%Gdz!EgH>YARqtzPK{+0KNzy(qrTli9`xn0pv?5R}al!9+oD)zEb9lUU_ z2ydGphu-;ol}XjuiC z?%@!V$~{tDHYEDmh4A%q&sqC*;rfI`zgstZjJciBQ0@&AO=qqDASpU&{4mj)1aW`z zb>RtqLk_K}T47Rj`o5@>d#VpH%E-Drm3!H!_6ENpe6ERdXrA*eld28no=UYcwBBn- zwcj;S1UgA|&xO4vRi}HclY6Sudr0f}!c^{M<8`oeq6n{_B!~7MIb~9{q1;m)W0axw z0Yj=I>nDk5KdD~7`}3Tu*2z7b-%Aax$I8BCsocxx^;^S#9TV%HB;ui;kwY(STx2sB zA%=2Kb&SC&1jSnuZ*!J=VA>`xk-e6MmS!%PY(62SZ7kUq2z|MG3BxsOj*9LN?-QO)KP!iPpLCei zY$)|qAH0--wOey)U01Ys)6WXu%%77(FGmw5MW-8`UFa$2tPdFx?VtH`BJe(mPVB8P zIe*ho=BbV^%D7s4E>G9#;QK!(!mm!2LmzDM=3KN+A*fv^HJ@p9MMqwpETZQ~ z^johtnG|g(@l;0_WmLUdGxKe@Xw9H0c5bqWZ~b{Wv}9q#q-aBVrxLB?^8VDM3QSjs z-<=%04kn8xJ`!7hUO47Xkwbm`XHBY3-xGBr4=FupCuGtLz2W8#|4t*?Id_V1wUg*W zT{Fj;J0cp&Je6o?k z*O*joDEOdy&`zYaY`y3kI=7en zJ^A3L#jIrePyLDrZ2XoSa?YK5ySe?*Q1+>gGs@t4bLxabR2G1-wXz9pjj?w3QmPHZtLJAIGTNk2_y$D3296tZ$Vw(ousU;e9dX!YLr zP0BVDe=6C^5PO?ym!#jN(+NM_V@+cDuL?&G$u8e=#-!|Yw{`Llr}$K<9hjUM~tHRazYjWs}_Y0G%4dtKeSfdQG|5;zPHnkgnO?b*5kVB!$SMD&EBk8Jj;!kyo z52@|gklI8to2`-UEq_4x4v_4%{q-hg8;U>G!3Oacy=Y9f|G)zx;Qe(uRMFL9QnsP^ zQ^`(FP8dAcWaW0y`|BbcC)wxrcbb%)zC-H7pDMMJzwk05I}-nO5pDhrIrQG!eI{ia zioap9X{5b3mD-uz)(rn*&A%by3%)If-dtNL%dGaxCRL{giOz3!aja2=w&Q`M>g0y5AKFL1EnI6KltUjmnscsNC;n6u z`IMpdzv-*iq@R23gTix(REN(WG^yH9`l&vQDMRg{<9cSZHL|^z9u&UazbS_{9=>2w zwxRS>$wp{<9I3%yRUoq zS53+`lzuANIM0A;@nC3Y<#y!cZ;9xJsdDJR^35h?8%jUbX}{zz8FgiA;xD#gs)%1D z*_&V7XHvGI_`|U#6@L`ksWB@)=J*`ib37W=iK|nEfez=mpZSQRBb5!R3B@Ukv3f!VQ>@4O18WDcZ8>#WFKx_ zTxKpu48@;HwldTX8I$ep{vF|K_+2@)wPC$U+37o^PW)+7JGvkhe_6ThZ}?pinEiWl zsM@>Bq-;a+r;?3bFhIXvGP02bXaAlEM@aVPFC8-}JKb%a!JlecA5z;>l}hcb+>S(k zPeeTr$)RJPT{J1%Q2eP5HaOGZ*`ShrZJOWkB+>k0EarJg#Cu8h+aJ!q)7&nZE?Y1E z=;Yt9X%>TxxBrtIs@ea#NzsPlZ`h%RPHO*Ya>LH7S{vGL|0m&^ z|NC<2i|yM?sy38X5Sha&p2T@IZ-{8X8pRHkRO zMz#CO!@|@0M{=mU+Z#|N8)jGkan&hVp zvT3xc=e1@86?yABBI^8OIdp#W$0k)9%01OlM#*P7rmJGmGcg(7$z4^mJI0)UEaG93 zz3;VoUp2Qo8cIIMR_RQe*r&fcCc=L#94r4s4%NT(nn~5^yQ5C@4a@5yIsCWD_b`~= z8r7Oj#JTcMgzEySUiev~N!5m;PxUbjrx<)IwZGxjm+ijrC&IJsKg*%`>99%J>1OL> zpX!`n=HJZd%l2;j&%!tFPvy}5aJNa>hQd#Epiu_d{?rbc)kfl<_opIoh-CY>JI0&a zA=72+q@QZCpEAg%+HPu`r~F1q-|gU`KNaECencxB1ydQ`YNN%kv; zE}4{_?zT<CZ3zlny zYQELz_OAbP;p->eN9y0mxo*7(9Gdi({AHL%sODRZZh!xui$LhV(8aH-cbL>|C<9eT z9A&sYD|MnF86Wz(gQ5Q-!c~7kSHLbmWKy@G5LD?_2HYWox+7J8A)=>A_mz)(P3lhH zFLhE-8{R{|X;OagK6ElLX}>nTPydC8`~NFl3j28ZJ?8dHLoo>5>M{4XlDlPcc%|7) zG_n)^|0*0a|B|kTy{E>c>~yh`|+SHgFWbe~+% zWm31H6jU8`kb-f#>hj?BoSonPbAKfQTOXrKV_&{%Qn#TLROwFsGS86i;MT`P_{G1b zt7D&Da<92GN#8AXQc#uN$@AIzx;1eUdGW7B^dRY;9e&fKZbK;u-Ki9$UrPSg&}Kqg zJFUj2pMtT2e=Xv#Pp3;{Z`*BBcDmi!rJ!|`ENzE<-gN4v+K{B)Sd3PP7LCtV5YP3nnPz&wdR-;Yx(aYzWDFxn%T={f6d%p zX($DU9doG0{+(oak04tc-i!ZEI8Kr5wmB5J?9DG zI6$)7Pj#4-Z72m*AJCMc_p($9W~bYE;0fXK{v%yR`~8GT-G)+7rCS+#A2+Dm?fpmL ziIeW7+bX`1>%CGZ1vTj%_NMmA?ELn||55mw|0i8cyVh$`x1khN=~jl`XAJ80H~&u& zSnyrCoVL5kq;5kgsG1gt*mJE^gx8N#vqf6)cCcP=w2+E4TdzS4nsoGHb4O4yX1fT)c-jh!X--dspYiiFq zX;O8%*E%CWRrV<-yDw&*VVC@O&}K)(kSEY#>_M9lBzWy z-h)^FNrd5t6Q2C-+pR0+oQKnWL&099rcgDO+Q9WbgkL(PjTk7u61} zF)7R3+@x$n*{70?^9_Lu#$>x^ z|BLWMNcNgbFaC_V9Wvc*o$yml3{0M6HR856@-M>Yd77@QeQKpi+37i=^9y0skwzJ3 zr`3NozdqA!B!17+BG5~+d)l_-T((a9sZIha!|dH7%&OON(VkiMs zpT+QL%R{mT{mo9d^NsHb*QccW{j>EZb*I~{6M?ETfJkl!tw3%_xBJuY3C}zKhpw}I zu*IZqLm8+#;vfT~v}Up)-QIWpkMOxJOZ)W8u=4QqBw&Gzko`8g3d zLAn#``%LOKl!B^bjxyrjpE^e}!uG96!QhGKMEK2rr3-E^t^8SY`=y~2gzi)d(yu2| zZD!3G!I8rI&3_fq&q?-MvujPtPTwtcQcx4#>bWJ?)@#*lB^tM5pZ}|f@A~)rVkg7FfhrBUJOP zMz?e6^TO3dx(~g3!K7|OA*ed&D8p{LYOY48=39+!ch~d6^VYx7wYN9T`8jhtWx8&i z5LC?!M0k7l8r1E5>)(XW`F*NXUDs)G(f(BVpjV0PQL-}!wJ2$Syqip?f< z8wx>{Zp|6(5p-+LJOsnv7vY*8&=t7*-ZQB?eZSO6L47j zCz^gJ9P|F2F2cQa@#I{GiO#2C)#o#1|Pzb6HI?BlVkU`zv)jtxxi=_Ltt4B@hHWY#? z-3V{hx=q>n?Z5aV5!m_v=u+HgFPhYyzF+Eupenp^{$bdK_J&<9Fq!3=7!2nDHdw z2(q>5ef-BFzV1KhTHNcmnUtMAq|!-2)#o#1D(!0ca*wA zF9=VS!$Mc&{>VAS+-_+o235k9vG>qbc}5VvWl;Ft5H3o+BwF}RL;2dH)h3mv@0mJ5 zs0nb#+Uq5XM!BCvi-3P7DX)0fZ&JCTC{!s|B7Ep%m2xg+XtrQ73zHkmpyxpnEy4~b zDL?0LHK{z^bDcEA=|NS9hwn&^tw#79p%ES-%_75$iwjJ%omDF ze)`ntPLO2bI0oHS2f`+G8%jf!?&Nqx?-nI z-JA>8$w8bkRK+(<7|eL0B5AlrcZ4L1sBZ!3KGt!-@SKJ4=)4^!g{M2NlZC38gvrw=MudAw zv+!+MNWyo1+HO*~p)gb(e3W5$+X$2LlA)s6#{HyO1m?I%xc6jl&V}oQAx<0)TCAk_ zbusLAngbLs=z2$pM< zCrGq#G`mT8t*gzXazlBjQjQOkzFDb{_t`0Tl4#*tu!xkuc;!Qr%F{j92}ISrLL|8> zWKg-AL<>&`ly{%GYErqOL{uGvkcjaGsS^{~dG00A!nbBIDc|$Kl3y{GEa}SiB5~-{ zq2?UQ2+Fl#PNGGi56bIyylGOop-jY4D3yuyJI=D9Or*KALnjhP$iyIN7U5k>NcjA9 zyG;r=l!?QH7s%m1ND5DWa@Nnp2x%5kcNGagUv=E1@bvvtClgg4f|N0Nc`6gLQywGH zA|8eEea=r!DmRpgD&@+koR%5Z%mdQ*JVBy`W5X+?yuNqAx6JLI>7MIkqUyw9^3 zZ&@nMvs3OR(ZaWE87bet@qkI?hB8rg9KrxsHD3iAk7`NcWGHGr@B2x#2%Ld(|EuRs zDmN60P_B38O|yS$jzU3_EW(?sN%x|8v+vLK{;89RsxTjZh4rMv8sVB@VuUn{=&a== z{F6_Xn-p#+5>+4Kl##hR6^Yp?kCA8*-w)*nqnk`B|FV&&oFhMEP@VE7yrc)x<+(XnxY5ayN+UquZmKeBMqubPV%Ly@Rbu7vr}a*_Ba$Z(NDO(^+#vk2F&B;m`?1WgJz6pAY0 z2u06dTt~Pz%}KL}egffFA{{1$r#r4QOjMmq)O>Cm!E%lA7>O3~pof$nZ%&w0ZYUE~ z$022Cema%r*(pzuXyJJIHB#POSMh7+_Rn`NhNuk9*Q7ErJLOIiEnFv{d}Wo_ zq;f-B*w=(OLB| z4w)G6rZO=*^U77k}EDQ}s($E0#YnW#Dr zDP!}^sZ7jHxsyZ-R~X9Q_#|plxuHx{DObkk{RWl0Nwo0Pyg|z6MlYLGp1y(|Xii=j+ZDObkk+fsX|=38Z`P8T)QOpNoBXc1^yL&{sX ztTCxP-E*B##0f=Jng=FlJnfP^nrdVRNwEmeT}!gpy&f?)1Gt@0-+}?sv8+NGHm_ zr_MxQYI&0pbo)uN2&{XPbVrt)F{#^74C1(>N^i#l$?t}rd>%n}kR*%nXVATA<`+5F zt&@YQX^6^D{NMC-Yd-r#NV15&TTi+dU#R-FxjZqHgQ`z$%1E4!^Mm$hHe4e-Mw&%@ z$p#Ys>EU{l!VLwXN;m@C;}~I9Tyjg%2v3k^;W!Q9?Ymk`3QylNb&_ycf?w+@8*$u8 znuW{nBjH5l7Uq3YaWa@cJ|xR*2w zUmJvf_G0CO=5|g)VTgmzpi*VjJvn(|0uke=N)5?J*v9>&Sp-)2NqEP_)j1cglZM0Q zA=3HrhmyiaICrZ#F%cxqBHRPv+fKY~Qn;ZwRDJAGM&UyjGR#hSghY#I<0ewRV(&qd z$_?eAO1Ux+52dz_?3Bkyw1}6#Mar)>TrjCTefQJ}L`{-M7o7MIEqG~20vfPutsL9}Ou#(ZD z*+bnVT6jK&a!>zTCY2k?MAcCUndsS&I;EmHlbBSl9g>r1;oH$b%4c@HmviNMp-86= zGkxaKoJUOR)(lwuBv}Mr+CsWx`%jzHZ736m9fb;1R|_0LxMoucl4cP;3gPd(eZ{13 zLy@TZ?52##7p5XHJLM4)Euw21N%^X^i+{^pu%z#vI+3Ue^S}Xv%3~y2#4kbloF(f` zDmN5~D&@+sd|4{Yv-3PbqJ^VnD=F`C>@ul5-E*BtR82%gm^(TRDtD4-;ac28%HKVI z)TDAlk*GQhDFbtFDiX8v+)bi|CkEwf4qwc*mzWnBaaY@UJ@;Q zzHOv@-mdvm&F!IvLJ^0d)X=;@_Kiu7aFb`{G{;YEnv-S`7=ZAe4X>LNZYUI0!iRst zA3?Y_%}KKezqg%)?_Ivlr113pQzsKuX|5SvWv4tsqD8d&ZBoAe#Scs>Hb2kTNvyF{s>0qJ?YD4pRPpYjw_*>tv#8CZaMl9~$6h=ee6i3(p5o?rZRyRBk8} zRmUME&4-Tt_$P=CT@!JnG$+x*_ga9IzwB)_soYQ~Lb)o<<98=_(4Yc>e8mglm!|NSa0X?VTk2gU>!QDcn#dsy@cybDM8vDicSr zT%$ZfqD6Fmkd(jm;mf~cE?Lqo*U3cH*~AfsSDKAQqdZ2UMf?bqFWJA=q;f-7A}!X}j)%0$(12$>j;8&n=6(IVdZ zE-8O?LwC-V>t*85OhnDVFgwo^Bw9F@g-H28wd429?VpA+QFRaqE^GM+?FZaEs+eo`y~`yt!edcvgabi;LqiK=Oc z$nx;N>dMyqk|IcoMR?_2l6`9PC6lrZ<)P{`oH8zN%sihjX}3mqgd~gT1?b-O+QNs- zWlFkkoitRPNlczP%=EjK5p>5$vWRbck94no=?#;*4P~L~_=7Br@63FiyAgCJNV0HL z>?7UfpEaA*Z72y8GmEixswKI>4;ymMcz-v8{ z4929!w2iSxY}2&nPVJG)Xxm(E+B7|rHi=2wq}tl1NqS{jPC;M+0Re?&<&^tUQPE}P z^5B$9E|*L08wCWR&-;$M`^>Z54f7fQz|H>h%=3KT=Y8jy*_n6VG3QhUcMu>8y7+S7 zF0C9=$Stc1GHy#1gxj7^TEF49shHbD_Dek=3mFrPJL!byQ_A*B+k7vq46=m;VdmkD zR!A>Co4M(!I3NoKV+C;gBt|LZwyh@)o=`!@OMWV@wlgoz4!9ARg-UHDfcFOg;?Ni_@tW7XW(YF??g)Wn75yd9v05&tG2&rj%3NG_`;B65CS4i72cxbd?v z@-cuG)GdrWDtJaAxoyrJ)I>BDkyUeTisdr%!pJoMEohUQfqZ^0=((F+S{(drM#mvu z&8;sDVy6NI>zW{TieeBz3%Y4WZt6R9)5#qaMRoxZRdefVt9EX#ki(E3n1zgz7631( zGb#j^RTR--h!sV{>*i5z2b|QzEMOK4A*}$eH#RB+msJzduQ+m&vM_QZKns-}jNC14 zOd+}L{^_75qH0cd$c2$t1GHeuyaeQ(;XYfG?Vqx0qJ$h~*rr=*;=;(=0a_TDXXHg5 zdWGb+IbT>!55zFBeO zOu!Df9+-uUqwN5GCdc)8Wjm;>qKLp*HCMm$8Q`><1G8Z8>;UjXx^RWyw)>}pnkcE} zOV^s<1wv$+4M{m6Hv+U!S;NSE_hc$0msJxHIX_7mlu2F<(1JOB6BkB42GD}~R2Pt^RZQJ70r}z7BMQl7)kJh0;^*d7mTGP%=loV8TuUF)}Oe{a|J?W>o)w&VRB zJ3rd7Hu2*XF3jyJRpM5GE#GcBQnh5oI=0E3E6i3_bNHpdikUA{F;?{pj=rfD386=F zo!8w4{{g317v5U$=e*JT{&lyjir*EWqF*)(f+`xOxWIm^dHaTmpbE`#CW>F=BBr=n zZQL}A1sHiqxz@v#d8_B)iX{SM3fXiYUdpyYHq8f?5v9sq#kTUW#wzRQ3i4A_y_|T? z&s)J13>xdGNd=K&L8T5Ikp#02EaRs$>{L&Z5R(-`^WD4r{?$mop ztV?^>6QzEJE68(M`+dw0YP;dg2Bx5v9?SR=iJlquWfDDYjw`6`30+@d(NVkhpZ_XT zkmr!>`$=@~h>aw=#`S9?`gAu}ke~7LuQMCe@58wd2m_Mnm-l~z?UktQjcyOJy$QAb zT;GF?5ZM$|eUmUBJNhl6WJhme3Tk_@=R-s}IQS50MMTZR%m(Er;P@ji6kkuAt~+b&oR}r1a-MLHxAGKFJi+_FCU>GX-U}VvH+@r}*M#mQ@tJ(C{6m zpjampzDqKn>;FBbAkP_BxPqd`Hh-UZ)}MZgFoz#{nh*vDKEt*`{jQzj3d*Y2rDvH9 zYQ=ofb4)?e$5XZtoBrVESuE7jo@-n|t!QuC%4|^En@YDaKS-%f{{it_7WzY`Ai~1y zTtS2X(V1=x(l{wg;EL!fa5x{BwUyw({Kn z6Q-c(UT&|F5kBAhD&s-zx>EU5;(6rA&zORucSgNNY%Y2HoGB<9^#fc%gw+>bXEr|8 zY}6k61=*X<#{81(O$P5@F+vnQdzdRI)}gvzGaD2uHvcz7iHiL#Q&2C5`M$w;&^X*X z_6D;-o_#OA$rO|ySHtf}OQ++1Pn1!=x0r&);qU}kQ1srWx0wx6I*jiSKTS#7nS$D0 zp?#MrsHJBo-xUTW%%@x5V+x9HEcydeP^|ovKQaZiD>wL0OhKMAuWQ}&Vgh9AzOvU{Fg-E>-{NHP}}{7KV@54+U271tW!HlT)U_=`L0~D z#CK6m9@J2)aUVA>S+eG$8WRdH_FDCPfLk4{L>2dqV~hs(+Cm zmynfb3cS@BtdKMp1n*16L%~HgJ1t6<-f&S}HNiW6p*v7=CQ1USuYojpxhwcWnJ|R} z)9z@V6(s~w+X1i31oxKl=%T7Q8^R?7QIQGnNtBL>l6~+RKPi3DyeJ`vyvzHz70P{1 z8NAaCd?AR`oCq!>8Xy9E_Qi$kYQ9{1jXK&7)j ziB7^0I ^F$&VK(^|dxVCte8Y)Il#id70R9O4_sTcyJZ$y`EKE&i}FGWbfgTDe{o zr8ujkZ%O4=Nmi+(5LU{SLUbj=Qm!B&E1zIkBo!nCk#Y?r=rm+i-UbVMf|P7^KUNA0 zPr{f$5P=!lTtdDw!(hQdutJ$An-e8uHP{797G{p%l&q6j6x% zlcI#I3R;UPL`P8xmynfrYAKh{m`LAyhD!*dat0*mA=Dis?eMH2BnOQNzq9Z-oOBeX zXTpDJvn^bpM2feL{7F*gt< zl}PJE{S-*h+h|P8UV>>5AR&mz5)&O0$J5%lgfdYd0(X|BBWH3}lu)w|barqn)F8vT zPA;J_(Vo@CQ-~yw2)IWh&1$@BHwFC5&+i zbw`H(IJK(1B1#BivU!49?JvB-r8MhcDo(yiAub0^atZnJx(1STV8*pgaS2(Klupx# z1L@bO6c{$cC6tNWIZ;AmwYh7SLR>$8ouVGdnWGS=BIl{prcpaHdNfvd_lXjMNUd_^ zR!FKks^Jo9<9M{2GaF2(JG5TzTtZ_aX9y&Sh&HUN^`H=QCp>9HeXJLk(6~SC>&+$P zzGqC75QJyrZf8;f5(|8|gdj>1eVy4TM85h0_HYU1K44Om5JXz5AGboj%8UHDgdqA- z0;rYGUM-hU#MBv(AcaV(Y7gWR%KgNdAS#7r26G8P9No8pi*R7B9~AmPK6}Vd5CX_P-D*}a|u}$XQfc7Cp?u) zXiR8b)3}5na(YAwC8wz(opT}gvxhUdgt{XkB7yNN5qdu z#gE6tkH^K2C&Z5@H#n;n!QV3UXa3GU`QIj8)Q6iFtJuG5*uSf9Q#s$bK^kA0mht{( zxO5Qp;f9swTcmL7)GCR0NttvXx4Os9jZ#BYq08nwxcggGJ=hMCNEz8=wuC6N5|zxRO_I&Y$X-d3N=6x}L*f_}-6f<# zRzfN1|9ZdA;m*h5<|kMGb6wweUE+N>bzj&0e7v8p=i~8wJoJrJ=?DlYNf?O91BEw% z&sZpl2?z)n=)ji@l;lK+uw87Z*a@l);3a0vOSIr6J_<@gbp}E`wy<3|UfIb&xEH)) z2!2Kg^D{iq&u~yuNv-oZxo)S8-ANl82D079j~;h&JI*%JPqG<&D;wrp>7l=wn!J|` zk(@xtR`KjMt15Vr2lFBu^gfJ~Yc!mktbcoh)HR~udl)d^Lj}D7DLDaxfQ?{zw`B+k zuG?AX+8rbAmW=Uyy#21{V=d0{ke1?pcDv85(hcvf+nEU%G#2Y-C!L7aRD36Y%k!}w z!+6L)>HX|(wXM=ak=O0Er5iN5>*geVj?z|qCwIs5u@1+0$jhRBcBj5p>5i7`c4w>& z8o$+NCw-3AQhcXy+w-wL%XrA!yngoc?XA+&x3AkBJGti7Fm=&eX8VT<=yUqXKRaSV zZl57MLO?Lsc8i99fShRK&)2uP63Z@TbP;H3WQxbvzHnn#xWmsk$M^2n?IV+!?Equ z`_dl=lp8)i6h|FUir&`ub2KoFJP1J%KJWJ+!_Z`^9QYfE2neYN2zKfj@7}8o{*tJB zGz1$7d^F$xx^hZGV2?T#|AEl;%Kp0dN!JRn%d6OYyB|zPPe3q9MMXzOMnQIrfI!+P zd%cmfmfi+!z0JxSxs2-=zC;)sCrdR<%E{&Qf76S0UgJt%U z%M8j!y6y&inlbcr+G}d1(s-3gZ3)1C@2@okzbItDzoD^F|1(Or(SW>eeZxfU2C|kE zk9LpA_Z|c+n77E?=*gAJE4++Ew31}YO zq9i0>^J{wWWRxL7KnwFbUqFSNH>%QdBO5iT&)h$z?6#(UWKxZdE3T>M|6p-zBq-xx z9?>B#rueWn{~AfY3BwbCL5Ix}7`hXfVxx*PYb5xG_J2kN*;$lPcAGO@W9U9`_Z;(e zv)j3+uQwIFlDPizmcRMY=6klpN4FQU9Ohh~HHGIOfC>}cdzW+n9|nPZ0s>JTG4e}7Xue^!Z*25MH%3y(rYl@3*MuKVoGCGv=liJpuSr6BE+tss2{ zeag1j>O;MAMFY~xFJ8KO%?>+%o|J~_R$ZOsw}ji_>dg<}J&5ojgiF6mIcVK}^=%^b z0#66(E2x(S0_!j!0KZMp3z7JX6pJDO0U-c=cg=1i^lEsp=N0ATQi z@1uTjCMlVy5;gH6h4!oEtAz`_&1^GhPH8pj$OuaFxUz@d@rTQZ3pZLK^JAC=MY+b9 zR$qM6hOF*5ZDX;-{YN-|HVLes5~zAFhAU z1RS<&;~lCmQ0hRYsJ=mV`6NkK(@5L-6T3E0i@#1_7y2eAJw<-nb~zxy9%D;M^ph6b z1mr)tn6BJS4L1Q4FcKU}btY9`PW|V?Z?V2eshN8HqV-Sod;KGIGwFkStFqAfX5(M+QF#Sb`I*6y- zK64k`vH7VZX;^L+oi8wd--0RU?dq-@Cz%LD)>3;pcBSxoPSL24`Tr60b1MQmTM(=@0<70Q8kv1}B;@EkH-Dg4=fya{evCUE60Czn!CSD>)*s^N3s|MR235;(Eh}1 zsQ8Sde+vRCCTjm10+J(fKmaL%ob5s+(eio#1^@vK52=)u2`WA}{M z2J;e^zUGX*NeBQ6kDgu`9!2g*G8ps~E~3jM-a_!w&p^4=2kCFavxRhmcgT$y zVQ&dBV}QITN?q&dZQ<7c0)zudXI=+Vp{3P72VDQxg}L^77deMrUbOb-wExiKU;D(n z+Xj0j*K9}@rr#DBFfndef;uxI?Ayn^ov!z+G&m!$?02Qn-+xQ{iTM^Yn@&EBSA3tZ z9XzBt)vG3b+SGpchHu%i<*dFXT`9-6=B(}LXFk|xy0Haael+Egnk{==PheldVe~CW zx@WoVp+}yqP1^Bq>C1rzngi27GeQkCEqSv)F`}~-l>Sth58-Ooe#Ih>|BGky2pv_r%zNiaYkZh= zTDVRN$+OrZ(-v3N0ItNZE(#HDshuia(WweGblPXMuNAH5r9U_I@L_G_CH+w zg#-k*t&7psz4ZmNwd`hFT(Ok1J~iE;MD|_j^3umpr)@DU1Xb%>A{py_S92)G6bMw{ z+C6D&nQLeH$cI6>upT}I!1Dki$BTRimkI)K6VRER#dga_x08>%Puh7IO#plj(5%c_ zcN}vLcr5N=cko)qr5>LhyvU)hAB5!_gv@Ln?~mDCt-8l7{hiR-T~f|b{Ep$@8e}&K zY#&QU%1adGvX(u6`ie^Bjq_gFfmWIg_GaY^XO=S63AYiU$@x2wb@IoZ~+y0XUI>5<$&YkwCn>E`S9( z2<)IDv?8eJ>L@!{m*u5OeuCE*0Y-tv*l8ft2HZ=4uc=AJ>NMYuM^Vlaqj!7$VHyx| z5qZ_J*LYM3$^Ty8MBezl%DDKQpEz>G5U1o3PVq1(R7I4)iWT5&$5X3s5#Ct9|aH|21 z8A5fNE_Mu%tD)w-PLdz${cRpYSO84~h>4fh1YqhvH%p9?4EflHOi*fMaYkTHBOt8> z<^Z7n4c-(Z$}S{5I^baojR8RUt43I)ag&Wbxau6szSki>Gl)Q9X4akSp++DH zGy-M4deS#q?G0tmJa_!$Z{W7=ITzXKCmQAU+dp^3H(dX4S+bu7Q~q_E>?gm|4XSzl zbA4Fa>^Dweej_lNx(0FKGbsNDy@jh<{)-I)mqZU8${`Z+)g*pLfCZJBf-xRtZ^HpwJFu7h%poso0HLZ)Sd=7%m z;?^dj=2^Nvg%WjlY6l{o^Lr`1=Y4K=Bs5z(WjZQB?lP;)fu@ss@e+e>F7oL2nqQ^M zw~gLpdM$FsU@ZNcUTr|@28WBf6hDRW{1kw{gkUaA2!_C206RPjOa_D?>C!*|<^no% zvILDt*c&eRuVH5+z|sSlTfX2{-2a>p<_rX(X+U6`U!%&W$9|n8V**3l!UJ5tlg~~A zXrO6;y0opBc}*nC`c~&G=Dqy{Tqhi!8xKrokc+aES7FG(Gi@1e7jKJm4+ImJc0aQi z$?%UJe(XWPIuRV?>GtSz!r9KIL7p??7F827dsMoe_n$Kdq~2Dy4G5^sz|V6 zeh?r5?t%})0R%*VHw@dg01&{9zzh(K{uFsQ^we4ksVl^I)emvR!NGy-x62P9)$wse zB*mOTyH{3f$a;INXZTOvk2=qI?id>d9luJ-&S6*EXc(4Npkx@%B4`@push{+^p?%9 zUO2o=K6{B)l1owkbnol5qu01qJ+Fu)g?)(JF37*`l=>dc2{R!-#@ZeDJrKZ8S}+^t z^DtyUH$)3!W&@q|6$DFz0&e(Vk)6dVMza4=PkqH2Y}x0+Y5}7E0fD*Q0K^LRWzwaP zXHZpA^=hr5dyh=quEJVNcHW;Cs}Cq{30WJR->pTp=SZ6aNw%BD`Wv3XiVvB1Yf|^d zsZt59AMmCt9g}o+t?@BGVYRoco$5hr!RI$y|78VFO5{U3a+G&?gbo94&~|AcfDs7IE%~`$Wn;xd(>Me|hMJ|; zv7WhLTrtWubhJGf3b%`^dLu}uJ0 zAtYYfU6uc>kaeJSq^IDOyhCDA@UQn%3Sp;jPXHxr%_zA$}NiM^De zHlXU5rrJKe!UH|_J*QE_hRMeE1~SR+J8R>bDQt|zY)u;a?k4|;;1>5R88dD1QoSZ| z)@|$`F80&sXdHRcs%W?)r3+afl}FXiE+^+fGP!p@y|O5t9|!Q07R-lfK@1%L z(}M75V9B%~rt{B*-$aEcy|H5Ep^qB@*8DCEfLXyNKmC;6RH_u#Y+?GcNVlOe!xGKy z-=wJ>KDZ8{U)feg+HT7p5VJMiXjHvZe5jT1)w>tDKqROSi3DfP0qr)!M0Viu&n)a( z6Hwl%L9^k(+K+;q?c({Zo6y|Vf?iM3{@LNq+A*H(S^sGNq6dM(e@GD(Ib>|8s@Mpk z)#L^i1A!&+g6ISsAOJ}NI$tdp1fVWpb|i3VSa$OC!SoF5B}fL80kAxv4M4y}h#P!i z?|uyo1$T&x($|Zc85O0vtJcOeD_*9R+1}Kn+afM;yyzTyNb5<6q}-bwV`9BO)11`A z3~z*IhFb`$3g1?aQFe{wjC{@F!9w#&rl@w#%5hVPSZ}2Ie9|iD}8_b8f zK~k6-^o2D7Cjd9NEOk&0{1#g{Qg)JVS@aw*Zy3M?fj~*0N-s&j)|w6PqZQ6ZNk-63 zQ>$LRpQuoD7dev3PA66xjARLE-PjbYaQebT@tuCri78`Wrb{s!16Z{8wUHcjZHJV? zPOddU=AW%Ld^4<$goXhc6T#xAsf(UC*LHmFouKRRb5qRamOim(v4cQW4jBUd6<|P` zPwuK>z>4`nobn(@8&GkLXqk-wJQmzUMQ8}b0Ucx8BlX>Eu;Ks%Fb*tENbu~-h_&zp z%z#d;q7t6%p9~FFJWDsGV6@-x z{m$Kbcch=wY?f>-YArGP$MBT7q&5Xg78 z9ngI6C7L>dPO&AzGuOcev;n6FgDquWx6>%NR$nUbR{g~jE8{-Isd+LCNh&0NNIr_K z_!!d1>wU-FBmX2W&JQ+!B|~)9kh7tDhbvbV30BMx;zR;SDUjT=TqJ&Nf{+)u@4IsvVc03HAUV#O`q;?MP}FYQ_1`Z<|~ zl>^WMLlOeRlUk&;mpUA@6rUFria!`AIjvT(Hfg}0xU{R9(t@3{uls= z1_1Dr>}y^5;PVOO;tXMT4LM>Yl7bEO{#E;`0>EN3!6h+-3kn1-R#0a{`B-xXXa}Yrc4hfmyolO(v~cZ71+j^qD<(3Jc;7kYmI`|Czol-3$*a9 z4L@p3A32OF5n;slnIQghgn2MWSP#zxF`GjvfFl%I4h-aG5y<(BjfVMZv%4<}1GvWm z*b~8P4}r$leQ7S?XGZMUn(GH&r`&nw%lgdSpnZ3Jk!8An(4N;h#6?eT?AE{a@fu(H zz9L__B7>?b$!~>EJ4hPx-EWKU8VIKIs=UIHbVjy8XKUeJvz)`pivBtip&IvKZoqIn zF6cJVVn>3}ND9P-Zh!-iAAep|99S_wh!Y23rN9H6jldeVFuMf^Lik89(+GUqo^z9t z6I{W;d^-g&*G|kQzQLTF930@KbT9w3z~e&9O^puwnv3~Pt6tSAQS;Ypk&xgt9Y(p- zc0|4l_gdE|{7X|=sVgWoEJE6b{{1FqifGZ@?9W*}-?y6ONc4K89n^Z0_&wO4?DFOu z{pOC7d}CLpi1GVKfS>eWE=&(%_<(m1J%||(z&sz-QdWmCa-p0oZiY4NHA;vasf`2# z@IA|1%smxwui$?rL@-U*1eOQEso;)QOP+Ti^)}kaN@a*Mgl8El?M73c(b}*MlATwz zm%i6>;l&olJy$mw_lYGHS*G|a+$c68>kZVo4mJlifX#tk<_~BU3aZl1YDsu=z;Z8V z8?)1d_lC`{;mrZ7n{j|vQsmZ0}6KWGWSb072cr&kgAca69h z(hfos0=OjzJ*pi?Mncgpd>KAjD<1n{FD>J0b<1ZFw|&>#3xUJi~3P6aIi`B^?@ zD|hQ%wWGC3;f!Hxo44{g?K-Dh>7DA^Ehs3OAzmj{_tK=t17_Glut>BxTd2h4w#sZ_>NF*ypq`2i)faUCuVKK_*upzF z*+Tf4zg!TQ(+u>V<-^Vg04v}EaRl;ma&o-B2LF1CI8Xlazs67^B5EaOMDfYMD5Eta zPyN2eCHWg?Uv2JXL*Lhm4a=7(QFC!>;m@=Su(;Y`{+;dnM6X>=Z>01(H_eW~sZAt| zlvzyAM3N#cBKC*BF(}M%(Qj9dQ9BoL9=}fp_{kP#!&ILT%oYa1n}iTsXuGsV0HYV0 zTk&(hnxhw*VV;qzfscgCE&e%Lf!o7$Kq~Bs0#Tuiv|n%{$WQkasD@^CPNc0*KV zl(|S_c5yR**4N8*-u-Sz6mx#0Rt7vic=e~fVWjr@wM9poOZ4Jt9lg@O*0M#&@f}r} zh)TGhN4Yl=&#NC_sX|v<%X~Y_S#kjDDu$s7mxuxw;m`?N%N#q)w6$V3mc-ZzA#N!6 z{W!1-NYj9$5Y;u6J`7QCyex9B>vw8sfq`-(p^uknEv2-28Y``^%i z++*IIcyni%doOz?x{8x!B1G0}NU^$|+0C`)qWR}T8@{1`JrQ+DT1yteW}O)t6dJdy zmQ3=^hfa&)m*xHEgkSt-DKZ}os;*Ka(7gZ+u6OmWDjKYqD#VEfu%SR6(Q?sXj(qUP z*8o^P2(1w7=p2RoMI`78@6^5!imdfT)IkbDjC}BAI$tVFjl7Lx9gUc-n1sA*@1d4* z9sK|cMbFZ)zLuA1Z!hHX=U&d)E_hn~@`bCs4yT$~tp%(ztT)u|4=)vRQmvNIbL|-x zl{Yw(I-oll9UPw*ILiAAzdHduWD4_PrjQtB3acQd5E=~-^DnIpzyQE?_$|ov!o`Tm z76pLcnL@0k7}Yhu7nkAXfo;)lGYdo!p#JIX9c8_hwz7;lj-|Fy@9 zyZo`{BJY2#J*T~|F^b`?&_}i7N*(c2VPgT_fmIvSHr4FVW~j4GXYk{U{$e6-l(ut_ zay(Z?#@;B)BQi$fm5cq`4C8AJ#3doU@|pic4M}^(SxcZS!fy+?>2!on`RG!ejl%P0 z0Dq~%e3&YvhN(i#Hc={|3e}f^1DzO)QnOaIE^4!7;s7Kb9GuxCjIf&l0?1bgGy>9A z8SJv~CSg5hlkg3FEC7Ne;4OHlNqnxiU}gSRtheAkm>saaxfS*n%o-rZ`U;izZ)g17 zT5NtF(0!K%G2%W2VhHqJH*TF17IAeq^xuXrUKJ+mc#G*UWm-D z4+e0-ULz>qedN56wJ{EnwK7u=s)eaEo}#K8BeRUrDX8RPogl5Z-0w+E*o;o>%-9s2 zWr8$(p83kVU;Id3&V9kt8aId?G@A9T6Kcy0erd<(N3MM~Ttw~gCHZG$0_%OJ(d|s> zOg}~Tq~~dlHuKY|VIE8tVwQ#u&bf1Y(i zHGnWt!)0xxg5Y`eDrdH{&koCY^0JCHa?%a9~-)Y142QyQ`d27XiF8@vm-8xVT z3czd_yG~$ET68zQpb5KH%yOi-C0KlXDDeKmNC31$|001|o461(Nq~Uh$LvPY;y~ct zLW>AXp<_ec3`4Cd5UiLd1k$1TO9PNzAdGW>O8Agc8Eg}DHBfDB zeAl-t-|YU$_nDEqN@^kJaW_~@2(J_QB1S)crMbqiM>gH}4ZoezP*~ZfJ_;oJnig_xZ{3_Ac*h47qU$y6amCzPECz&puqfo0P-7k!TOC_)QKH~5XD2WjRLBEHBP!&`W4yk{Qs5qU5-j3HRqFoG0W>rJI4J*yR54hFn&;T|iQ+GNGof7m) z8&Mw8n$~Nfdv;Fdik2V?g{hE3bmnRQEpo3uIP6V78$`=2zo*YA%ge_?;`?3Ej*O_x zV^=5%qkou4w{jXhiQqOuM6idSW(uwKU}rwZ_Y1!v0AJ}sXIsl`JIh&yKr;p&6JY4V zB|-s)Ey&5*R7Z+zB-MC8eGGdFz>Nfp(}mk2fZy2Pr3(wo5r~RQ^lYf+mMbKyNf)e` zF2o51*ot3nKfnRm4b5y6@QiKGx+%R=00;=d#R>i5^o|M%38B0c930GmSusd2O_&s^ z;(pH5<-U>AM}hD>WhHf$Tpmlo{;G2{S>*W#+y~alDS7%1P*0KsuZ?ocK4id%zGj(@ z_Ec8zX;@n#H@4l91^vr=cU32uE6Fa^QRB+z6)$=1>q^LJ@LfoZhm2u1O!vXb12(fx z#Kh7|YX>mGp}7S=_p6!10Bkg*O!?blaq(x-5UlU;oo$6k4W!zS%D3pdUf)ir499yb zNE91!aaa@`$_OQMEJm|6Z2ig-@pVeTr$=qLC_U+jX*=^lS1xn<0e|7ru8QN|3idYW zY(Mkv{wa%==yI!+O$oOWv~|ty{)D!QCyxBk&0o}LsOl0u0^JU3|MtVIRjvKSMuSVD z3UNvW$RU4$VjNWA654=DNM6kJ?9?_Y_z=n#t{%}?6f5jBx z0`(7E1%58O{^!^V2#-x?-t^*~xw_Es)d@a!uCoSL`o4AGwKvk7f>nn#f_is;Mn+I& z?5#T<^|o(3xnJwFvYyspd{OWi9s6eF!9tG0^F2j96X^^|>gTSp$V-qFN!cr=U)K7E zo`hxaanZfLjVb69d+l2~wIN1V3hxgDzs7gtas1>6|4a;FC&sq)5)?y7wdg5e{!H+9 zju03P{bo4&H#oxXEC$5LH~@n2n3t;x1dBBSOIjtw$q_={&qEl1@Xt}gAj6{*Twm>lm3<$S*jP}$2NO_!jF9Yl-`GyLWBb`7`=Feh&ojmm^$x8eu198*v+yM!4<&0S0Cy zLuKK!+QGjh85$jDKwNml$cB31J-n(guwsS~r(_703S{FP3zD-HBrFtEG}8|#5nQku zVlK(*>yr!(+p02q?Nmw4PEMJ#%AtnJej3=z7glcaj? z3X!VSHNMzxLP72f)w(lc>L#Q`+%1yn)xM$hiJ2~;eNt&hRqn~XG4@>Fi)<3l%1!pv z($jKP^HgovbJwpO-%SVclOp^Rk%T}h07VjRGysu=v-Sel;d{=|J?ELD!pv-7-r*on zv(hf5)ej>an%flYf`vm4k99MyyEl9LaSRo8YnLl@H$x7jP)xa}AJ%H5KnHRl-PXoT z=2{b9-l;<)3kip?a)Bq13pvJV#Nza||8ODqe#D3fi(q0y-5c^JUCqJZVqmZ&o)82R z;+6}bMdr^1u|tE!Ujp5jE!v7f+ z%+&w9LqSycV$Oo-%80042Xl2#P} zR!kD&)CbgHvz2sCB%~V zMBht)8f@`eF=L@s#7O>j>HC-R$j|@ZO5yz=wdl&6+DQ*lC3SlXSBrBV z+ywZOP6(`ppmah@UXV^WBNtcz39uyLZ!7`og1+)=Ogh*bki|g-c)}j z_B_aB=1__3&+Sqcmux+eMw5Jzk?X{OguvTLZPMC+B&_>j$3TfIa335R@aHP+s=uS~;CljI|pP3CcS#x()p*7m(XKZKbUqS1(%`5`nY9Pm=5 z5au0LR@Ux{ObzK~9X$E!VV?o|vuajE!@&|0m!C~HGSda4H8Yk}Z?nZ{!&9 zg8!_NzuJeu?}s^dIY^~VdJhQ7iI2WuIws;G`l>E7@a+GQgt+kF|K|~fmN}S+LVplZ zIFnPp00_)$2bjXWXOqb+p7PI71zNz19HI)Gv)HLXCXpJ*ByvXwuLa`aqt}zVp>?9} zRJU2l5FAPrX@26+-ZO1fAC0$_eSZC0DpBkJaV%&Dz+N$wQ~qhuz2d@rW<zMk4upk`^ORT@5*uxakKg7v%k7vnTZL;1q?Hwcj zD$n6zhNVW=#Z>+pVPH^VVQpx=7vCEJ{G<&3Ohh3ldpM%-eSHN9hALP92<8n3nfkzR zz_%IBDfP}8MeX8Aff9?W!)%X#Bm~2O*ACdNLaNQj{&P48;)&qlpg6QdAUVJ+eOlM$ zk&NBaEr*)xSP-3YtZb-+py^e`fyJ7EB@u=oqHz8UfD*z1e;kD2;y3^^1&xq$sG{qG zVEt|P5*nZzg8f8*rHBsS`-~z5N~8Uz4XJNu4;(p?)@r@N#Y1)g+*2 z5i`Y^L)6EIlxn9W*-ZIKN-{MQ-v+IO^ zK|ItQ$BG#F03hJ!STxz+;s=&wJBm|01p5m#EE@vmE)sqI$%0uV>RbmbhHfBK))!GY zLg#>E?NF3?n-!S{2q+`Jxt^Imn`w_ovq-*CsHRX2s_gABw3{zBVB%{Bl(`rg< z@q}+m@mCuZ9k-P+L$~pIX?yo-kXqfIF&!sco{6==m%EBg*$9)sE*DXe+>ne#1ev-LR>5%HCw0o z{00F!C|vlusA6E|JaDkZA z@LCZtA5!XCM=uH8lEf;BX4t}c2ZF#hzuEDickF20ZwG=b!W)i4ukC+R3lO!Q$l#Y; zB(&TT8(I#tg;f@t42MIvCf%mnR`L_dEdHAN<3BQsz3;Fg!hG1-P!H>CRuuZvn2Ppd~OhJqk4g zl>2w(s9cY*;7?P|RNZrFV2e#@)F;DpN+hwt{9+y^?8awRp0S_)(0Z`hS#8=s^ML7q z0~dNoGk`I1gG*`imtEJ${5Ax4M4i#{PyeZT1r=DUi>ecGu+n+|e_@4xg)aPKVTIXn zSYaF#Rv3pJ4a_+uSk559e75lSu)@Y3$k3 z7d0euJ~=}FIsaxNBmMLMU`Q5%DhcCyn&5m>3eQuD@Nl1S1xNlJa z`29>UFRQQvIuc+~i*aWaCSRml<*dTgCU!)@H4Zk^#q{x2t^dV_gG*ux7gP(NAK+jL z7jFhQA*N8Ds?rh+2Rp8G{S?~%=70|7!gK^sTL4|+fGUg!Y~kb=FWM^%7Wa~U-)31V zbp@twsxj8KZ8bAEsx`Rzw3N1N>p#uT`jN!X^o!(;&t<&197Wx#_S%`!Q@ z7JMMXTo+`R`+*E|F%Oo>&6LL|KQYNM$yyr<zgy}<8KfH5PU!xu)%{$;+f@EQlAVh}(<_m#`!t2rcGF<*!i3LrD`0v{ZF;o?v* z=Unjc^G`s0IAeIcN$F1W-CbE!K_PzG^^QZ=d2x(jMB{m!fQuSB7$@m(&og4BPG#J<1tTxWLS>;Gl4zuA8mBhI36qk*zn$@CE zW#Q40r-bjV`=zWbDz2i4@9u;6$rk>Z#6qAOfD;RqK|0~g`oID}0FMWL%POow&~KwW zyb+sK7%+j9ie9 zw+lWHfDb@OhyX8{jD1P$5(($#(6$z?Ycw&-DKi{T{;m}->$s!nN7Dnr=&i23%^6m@QB#8f!@~5|to6y;(hsTG?0jyz zMU4yJSBvnMFkE?DAz%mKxI#Y=S6IB{!@>NHU|XcdU+xH+OV=VggE-kx=^x8h6$utw zD_Rn5h*LoX(S{=-IB3KF?g$!YVoN`cJP6cx`yanCVP7-Pz6?x|IUHO1)M?s-?`qVx zl+5Ihhz%8a75SBp>)KUH0tN^sDT1ZHv63^j-UPT&rY!D1!6i>xLf5;$cZXG{cbK+`&V;apZ*tNy*OGWYOTG}En~&iqZTKew3&FD=4lE=FfrZ#j z;|1CQ+~+Ec9Ol6#KBHczESE?HBoPKG_yU?!WqMn{UKLb@5r5Y>Zjt+ zmZLu`i02r9pQPc+(+WG`41!Kd{yK? zbprSe&s^(vf;(Wz5UdqU508i%avjxfW4uo_Dc+}m4DlxOlT*_VrqZ^Eijg_flAhWZ zU>oz{py^iAY5VsMZ^E>X$vDOrNf$Xa@g|>&KhA5zRmU$@V)EEz93|BAy~ntjSc-o? zYvn$Cw;jV*_t8HUR@iyMwiARECPQI`1X$(3N!!j`yUrAFYX%(tcSiW{b|GKMh-lzi zSV2_$d6z|d{q6eHpPfz)EsAGkQ@*NIMzbX3(9+PEo=6E@>qtP8&>M2tiFdMLuaxHY zFwBUmmzF~0Il*`8AAIt% z63Bl@;CU;6hh*U&h%3bUmqKxcjM#a=oYvrfs~>I#=8=UKb-;n}?~;WPV_b;jDsDDZ zc1*&mR{vs?!6lJ}I6a48mjQnqWZ~lF4>JTa{eW`47w89ovjApyAXDoAkPFSYk3xTH zf=&Ty2U2zj3hJs)=tySuXgo+R;)`ZmBa`2}KHcW>7W%v4stRosYzFdjBkkay9l;=qcTLYz1NTMRs1HV(|O7`QU_ zAnm4C6rcehUjZyNfQI}>v4jDyA$Y9=SPZz_O!2+N*CKpW&sC|x|8-aly~n1>1C$Ju zZcJHG1ESAF6bzD#Pox(n_wRfe!ENWccdXRlhx!`@nk1QXO=GIJH$3}QWUnRYayf(+ zsaW{kny!|;Hq%+SkQBd>06)3HKNDDJnF9wF78-)U!WqrLe|H4`BEd{xq5k{uzidWY z7{-IB=m1F2KlpT2kzlcoU`bpdPBT(SGca-m2Uoa+jsWZwz=4IrKq;JcKn^_lu?ix< z3%mlG1iB#5xDwhd0`r2fU&ABPzDKlG?IS^kIg6m9wqu=q(dk}v#%-N@+=_~x{f-6%u^mm)wAzERmj@2zScbap3r&K%Fx!NvG7#J8dmN6s16%7 zYdU=2Fvee|aOH`Gme+-OPGAxXw=T0GfHsUv`&34VJQ>`$Yk%J?!mJlI&!!c&+5cB= z=}cPT5VToTT(Gwu&Mi$i+u8J!r;KVCjwzcWTx={*)xm>6=dEKyJ#BxtsxYu(q7bKa z2(}t%T{aBB4Z-YcQTLCv=iabF3x$ZMEbyK&W~~q)z&@ZKg03paYM?$j+4r=dg1|@h z<2ySNr{ch*kOE8!jlrbQHmY2-vF}I9=#?XaIXiEwB#MoHk~xAdQoFR}Mrrz2d)egb zu+YQudcE7yo+i`_?iVlId~!<qGkY`+?@Xb%@UI00{DGe*85cSQ1wV0t@HQ0U=_b0EvSuTv|V12~tWPoFDy3 zGpiwlHV9rcLl$*>!EpjkEUbE&&PSA3wMC7zhD%#lyVT`&Z+qH3y@%U>a*5Gq z%cPFRi5cgXn8X{N-F2B{b8LEk_8pq2Y0}kx-0N-~rVQJ+kxaA9l8o{*vD%}HM57JVT>TBz&vjkH z-V$yhAh>P^j`q+p;7Sq;0Y3;Q7W#n1!V#?f@LUFQc1#HC2bC#*;|u`^pk3d=-9V+P zhwX2iA*>(V^Z%kB^!tC+4|acFhZsTevY{^Dilkgkj$p;4Ax_m01Ub2ktA2*F zhF=TQUPJ1q)hwhYDkWuYE1H;Pb}^4HTF2zN!ENnC`SG>Z_w1W9J!1DLzM}2jnEoa3 z_+d}oKDo4)-W@vM^ZH78?KFmx%J?URDJ7XKY|MH~@qMEJUunZX6!&?u_MaLB%OT^}vC7PXI?1(Um;zeAD9ugN_0 zM-R%lw|oPWLW%Q5T3Uh=@`=qZ&kVvcHfk6q9l9hFACjs|*%E#pzvqQ`NFDxx+`^eX zVoZwR+^^t`V-RqTBexJ4u(9L>uREb)(38xv>C0y)K8BMxo9ah$x7}IL3Ctlk9G(Sz;`^|{8Y(g6i&qSR^ZGh7@w}zu7 zw`AAL_)Ev=0L8$>dFPw2|1lhls~sm>tnR&I`4D06LESe7R8^zf~wMIS{ z@A(kPxB1yn1!J#PwfoWgAE2E2$u-}bIb>K*Bis;gL!#*z<&6ZkmD4D&LUap9~y5fRxd_N1|CsX)mVhb%X z%ZYwaY$4W!bao7Y0fiZez`aF4-t32_YK6K>^i+fkJ4juwx*Yv9!C|VkFI<6>=dB9bn$u z&t`w6E28{e)8H&ekhfSLkb0XRQE*X!4He!wx~e#^SRe5J&J;qL0rbn|;s91UROt+S z2BsUOHgxp~VU&Ypu;XArZH$986|e6;^6;3L7to3^a3R%a(cT;3_lo}+U%CF{3t(A* zp5{r9r%WRHkoqUN!59~_CJ)tn5{h4$I7P-pe+Tl7@Wu+v0#Kr1Pr0!jS}<5Ekw=%Q>8^=&v&oij68W z9FGgSO|%#e6kZfSR5St{7=0Q{wE?UlGgA^15D+lXf?Z@j3Q9tC20}e>cE<6_P6oog z;1xsgGeVf3;emdJgOW;WoyW;_J8kSv+So9V?LL0=xRcv)wiQ!_IJHBt%fN#1gbPgq z;ADbLRLa1N`14Td>)4S#3uq*$(rdV(+@-HM11u4ikCFORLL)-($GTr{N2q-3>Xdr7 zA&J@vFs~L+>T+8&8+vmcyEj^yNZhNzHb<-iDpisZ%pCGZf7He$_O% zWG#n1hm(V?{jD^VQJL_2O&*kcp~gvy$;kRZ0esIc#ZQv(Pec|1tpF5Rh&!I}zghu^ zBy?%QwgR;}f4LQ?Y7{`A69w5&E%#FY8U!qfBm`-N^GQP3V4!OuztM#tV2*n5;n?<6 zkYtXL3N}FsLg)f2Ld$e(YoHr6VS-R?>b`q+$mgnTwy0BuF!cVu5f`aVBTS1PFx`@6Idj!uqILdcoCWD zIFr|E#T7Ou3L-i`0R$*YD_K0-qnmN z?EWN(7`ZRRhKjz@ysA*JVqOp@6u_c@CZhQ@PXH*u5$5^#f1?Ix;tHRPO{aiv;NR5& z@Y!rr0J;LGDG=a=x`9G3{h%pjHMRGV{j8gvj_hbXtMwziBYabs_CAhw)srqKHG0^4 z;+gg_g^Cu6CK>djU9}ZrC#2=wT@qSqv%Yvs5}L=lP>CKe+HdDlux z5y-W$1bv-iQ0-Q8y6TXjeyODE`th{2DsMiUxN){}@rM>zj;-*Bv*F5l)rT7 zN6yF?r(bA`qI>1vw^uJt6OP^&LR<(GW_uzxcQ+}rR%u?=Ix?0kq zDJm4ad8#_sRFkXQu|3&%ze46>JM#z8^-}vj6qHnEUiiu7%}_t09q4TZB!GpTVp5hMJ|;$J@!c6{jmpNAm5SpAu?VG`rY4 zP#6*R0-(Ul4;QI%i~#@GaB#(BAx;#4$iig9g{pm_`%I7-u`*c|*d{tL+)~zv&RVOW zo3zbiw5JwYBNB>4)>@m0Ma#SNqEl1@Xu0HFG=IGXQjUibiu58zT+LT?WAk`lrlE~m zv)T-$l4a75X?pCe3CxY_-?W+HhH#Ni-_{Hr^_T2#PL?$G8KQ5e?^PQnIeX;0WZ%oM zhpCpTM|3>(KPXKI9H>hCKLb)+xy3*{Y;5C>_*C!A_Z(@ZnlD`)Vr1vz;lTo4+q+PNVq|cyF*%n)UsCTYtKwA05OINSiVdu}2 z(oo&1tCReea62p>{xflf;6V?^6?WHGV6)0*AN_M7V2Xf1u|XMvfSpYk2q@&BOfe`S z?*R}%dEa=*%x2^MHo6CX&eHdOTzIjIF-hi~a>n&G`$`oWCsV@yH=FX34mE}YeqfB^ zlzob?pZUn)Sco6k*Ul{KS`%AtnT4SVe-A4hn<5*oDz(^s6S;L!nP9{VVMKDe2pg)n z8M&$muvj0kB$^OsSm7E70TKuoD)oi#!+L?4$Z*BAX~Cf4D3~$OpGp6#-UE5aJXJ!R7)z%SM1X!$HRn%(KHmpV{DYU^NCtgb-~Q z2&RM3Y;b2=)J20}mrEf+{|MQk+(ZhQ+(b`AO^AGn`s%>abW$lkZbDUF(1ojsKjUlM zcJ{NvV^4FoW;7?%QRU`kmt?+*=~pKJr1>{qXWguR8CM4ga@CxIhokVi#xF zUynBUR6mmH*M#g+p$luVWm{zcu@J%F>@mOsdkok< zaMA^6g|@iX_yNnY4d0BatK8m|`7(OdXmSw`ZhBqlwyoqB6j0Wf`|&>l$_htC5ET`o zY^d)!wSNr)mP8POpu+hCA?zxUjbkRb_%vY7I`R2;KPYCC%Yl=CO1!R+F~}yFl@3;e zox;)W0U8uLu8Yv05vdpbZe(UKnpTv4N>A)WQ*Um|vt+MCZXw0BV}-qM((5zJ)cjO! z;)^8N6{X0mX==G6rBfP0zMtZ_FJs<(NKgFA6b;D%U`aA4y4~Pp)b8*kd>;zpCquaM zs6xx@@G2rLluJmvd=Nmpwy0H7^=hq|OA_XDnf{}<&;GGbT}rnWmC%uW4p1BLW3~<0 zi){nwN;#ofz-4F_P}WZMpmpPyH(NJ!J+jzj^hC*v`rZK7m7!mq=I$?V_scPUq+3iV zR8=90K!=L4q5g^PT2&BOF-M3~C73@#f4pkYA64yub#Z28c;VyDiQa$a9p6oka_{AEw7c8!fTjh&&yN0?p z`#7zsm9vfsJE-hH{9eLj!az}JaLu^1Og``<|3o+_?9ze$sWeO`#ygpzlvQb6`JsZ@ z+U#2@zUzkZl_mUBDTP2Q;A^5xSq!BVVsi)4SSeh(;YZBezh}?|v-q zVR!Ia#-$#g9lXe)t|x@$H-*+qy%T$y*~Kev+gDuce~DTtv`DR6;q9ZaZ|F$d{vvd) z#mJrw2R4wOt~P(;Y#>|KD!zU6*NM!OQ!kZ*d=hLdBS3I*sPfYnaz%Rw9dF)!+x2}> zdy(c)F+}HU00m_iCs!2;R?HFNgaX)AppbZ>!~a6}=WY>w`AH1B3KYQUge<&_DncwO z`yyJNcYr`ROQjH@h|SWNa(9!`Wss z>kt%YS6FbiX^<;8qZw? z@Q@;0c}ig?CZ+HTlv3!vj8*_}g8%^b+he)G10J>qdEokoxj`}vH+Z|q9B_l{A#U(E z;0BxLbAw*8zW*P4XC4n_|F(ZEvS)-0LdYJzW}lO}gcGKl6Hhuh(;T_tf|P^E#Z@c^v2QE`FWUw?0Qv zxkqy-fH5ET2D9aM!tcL6c>iylwv&AR!sP3PwsDfJ!|g9c5d*nmNKC?e6vKSl0;HCL zvoZ&Noe}UOItnaC-Z?_@{l87CY63=qx_9f1TD##M0X6EiKjwk%%D2Z=2Ou7jk`z7yqrwMWjpirMX7Bwe}i z55E8DrS%fnISwSSTcOu1WawJ3Ofi!pfYQO%aooLtCKsvjNsAZKN*baiDV$S29 z=XXy;2m9CfFW1}ljFJa$5&Bvl-Lv(dUdmkuP7FYw1;8TAXyICGSqRwW@M<6=_x^8I z1B1C@2+uJf1>c@!&#M&7*&zf{F!RlZasr<0Ggk(`@eS}6A;c>M1w*ffzAnTc6vNlU zpkfHEfdMx};Essj2T?ej!4)+szaf_TP!3IG7AB=g#VsI6DrU3Z6XDHSPTo@Dh0h~K zgr5qO7}a?O^5+IsCzv{FB~Xq;$?R;fQFlz*vf$9f6wsVMTx{}b1SC=l5^A1#qxyo8#b+z;y&TBQ)06Klq+dMGXBb8MX2D+X3Me zC1$;@!&1sN-!OW*0=+Ww><4Ue64D$Jmr!!B*>6d;jrl@LxtaDLO9}dNL!R#EQNB~C$ChEpUKPY=bbz|U?P_p?*C4=0{Blu%NpLnQS z2&@Vvz)7K*+EZq|^$f?LYoLB(Y({~R?*dkBa}G0Cy-4=WZ00*m9S!UK+!Q(b?2o07 z1#Dq^Zq1Ej`iQ95xx%>vZ>Hu3#R9?GH~%Q+be1Jw)!G2jqEH z_%|?xATR)93YP#(;lyPzaUJ-DXGo|XmF$e1Qbk_32(k<8?*;2^{g6&cC;QSK>L6Y` zc~X=={kwpwr2Q_qE@nHKv-@?)QL75yvOSqlbWj3B2f;u9qJ!Fx_x>h2m>MaLD0mHI zASHg_Ka+vkd4|L&g-bxcKta!+%fPfHA*aoIrugR zT*GO~wB=myI(bX@6JJhv1@@Fq{n_@l0?c|@5f6ov%&sIK(+y!&dOG|ZtC;*K>Xza- zS2l;qd!3OBjm-8mvX2^db#J3$LP3$S5GXQMZL!?qv#Vw!c{dD@r)4;IP$58r;&{Pkc``0YF}=^ zKzF&wtpB{qMGza5)YQ4s#ry`VMnhL(JsBHhBg6)UpmveEURLIjzO&s?pn&`}TrRRj zX@5KE0`kJw;)pv}B#;=dvuBsi=Z0|3W+Bng5Zo$ojYt9cZ-#(r-vf*tWuKi6LscU9 z=Lzf#JOlQ?3h2QAyMXufDX3#`TXY1q91APUVUZ>?x8z=p2I+@+GPmp7RUZglZizT; zH=4FrQ}k#~ec7gbL605g?~T>doTFtDhPm07#CQ}M6jF6;>sVTSRRn3$Wh|5VEW<`E zN9Y0H5V@y^$<;9YTdYEB$9>k|xDa9$wh|hMe~~97YzKZ|3cZ10IEg7V=AMcv+?X=D zc{-+WeLb>H<=2!#=btHs+8o=uf;x@&v=<&<+862mBKMHXv9A=QwgVMcBoNr|Kn`9H zCC#fG%-Jv`DhF`G!1$lb!Ni&n=8~|vxg-_^kY_8iN?mpV$&DKZ?&4wy1De+ zmpOr5lTsgkPvoH$y4G$;P)Mj>T~)O$+FW;B=wO98xqlImp z1_&Pm(*+=ikQm4*y&9t(qxre~752LOr=x29abAkzMW$RFR_Hw$=cx`AVd3}R^^uX3 zj@9rc0;SXzrj(v2=>MC(IIB$}s$8U|`@DkqYwPj04lo2pC89-R?7iAD@ZmEclHE zAwsAQJ+6N;bf|C=8gbF2ZX(fgohV?T;=6ISi1?|F;I!x=kRvH<@bUc?CvG#ZM}l? ziUG4hyH=Uv!pjqfMJ|(x;$ev+&Q%q8>#vBd`{*SpyJ5^BKKjVM{36r-Kid!v#;rkk z_5dyLyYY5jwSd%6Fe}RtunK2dhH%Azz)S;yusNLhAK6Tc1D<4)9m$?tW=!9e7#mCGEF06!AV@^^mVwurj6l{62|?qv%09_mvNX z-s(80v;=%Xc}e>+fuH|-^_XkSM5S_%<>oiCG)4{QcDbkRGHgy5_w%C)7SA!lwH)Ic z4k7nEG5OvV{smHDwh+(S$I_^v)sy`K)0c)5Z3Bc4gDJUzBDdw2IN7||?2K@hMvdw& zC(oYi;6GgfBw%#uWiG+h;JCQc{7M6mfGyy-_<|kJ+a8fOP{Dxrk9pvjGybwDm@p57 zkKP-F_JsV$>CyMs?CUQY{zNreQ3lS7<*s}qwHWZ~S%W|otwmzqrhcDS4Vbf8NHjhK z7YrQuV-1)#4#*8^CG`!1pZsORIOr#Tu?BStR62^kFKC0yNTG3njXLtl!jxS$QiX?) z$@}sQQflrB)Ym&DlV|q?b!MCBa~bhfwhY{35{5<7avro*v4cRM16VcYGevw-|75dO2 zjw39NQ2(In53^9d7ul&T+$}Jf9&k=zTgo% zyR#_xK`@0v|3-+4ijI#(N%L$*4kd~j@LZyNwqU*f5%qyBYXYo`{kGG8TjI62LR>@S zF8eC^bjl8a2*qsC>``nZmOJ@i>oFg9x_su7ipcogCw%J}?lG=Ow7yk7?)i05mgJbd zDN~}vck*5jkf&KVcTOR21Yk~~AK(-gkXC`2i2+CjUIf#@2ZoHDF+V2;p4t72#6ai2 zP7L@-AqpCS2;_|C&8rB^*)t>>9D+Q<)}cQa0r-PpVmGkWEzZcrwrH}092yIpjzN(j zsfi;4Cy*Q(@)^t!a`H7$xvnRXeo4z!BV6I(UJF^fIBbgU+lV~zVv*t--6`EoDbXn| z(aKS`l*Z-9M{_o;JgrlHwqmWOS>XPN)xwrx7U3=)RWRnis+2Eoh_Vwvf)PhxWC25 zo+s(cyq5&W$sBA*Re#6*emtGgeq(B-PmoBAW1&o8bjKx3GbVT7`u{9LVsYYZ3X1T3 z+uGjLt|@;M)25v*UFp4TzaMi>9X;o<1onLq3eqx-3eu%^yPftGVHd4(HdwHX?hQo% zTdxZ5spPZkjfsSiE1wwR2d6t==l!pXP*@z*9)3X#{ovy!`L#TjurUC$$%xm03tMEI^B(MPmJkn1#Q2`6t2x z_?x2PgfGRB-rj&x2#$(Yi;G{r2JVbyV%Ei6x0M3<;hOSt)fR9|z=s9r#p|k;2yoRH zh!&e$ps#heVsC28MJIZ=xev2rpJu&6EIs)L$! zu`d~16$b&jNMwV_$&?Pgri4KMOJ@fK$oyM?e67N{GYVVb&0p(8 zN&jye$EIe7lL*;i&Jx%dsL3lT5WCe&sW@p2T*!9OHn`EdD1>{KO(VDgX+-NS86>8* z?Bl!&z?@COUkCs_vf#m=3jmxJn7|Xi>L~eiJn4xAuMHUfnZZ9WqZkwp!Q4`BsB{3r z6IYztA8~~5*v5{kj?*GK!e8peGK7U9&&qM`G1QP~5Hd1m>#mCLtt z#U8hB%6jPekzXOWTOs8|M4zME>aLr7?uK0#NT5ZIy2;cIggmnwIHcU zU@i_JJ*X2P<_>^OK?c$O0SHqLeFfA_92NF=DD^go%c#k(?`@qZ8-uFGa6VepF$Ne= zNc?5YGI1!4_LV^nUspx7R{>Qh!%?1PRJRsk=y~P5+85Vwo9|10lv2ESFXc6c#v9mY zs*a487okJ2^4_is-qC(!h*vCPL~h(Y?p4M4glFJ1B3ER`bKzp^z^$6WO6@O#`&X z_k+(L%}d-<2W|@1c5&fv3R)}is?WekAQ$h<0b>DBgJ_@zRz*+KdOip~ zAz4N~7$l4Ed;!$pd}QXlssSmJAo-FHyE-5q^5BVwNQijIOuJb7@L`)Ds6&t)nrZ*T zRRdBp)qt=pnE4;z^8(X83wTDjz7#)Rs<>MGGU6PdmR|v9#&^}SGw>G$&fvD7(_?7} zU+N`ujem^8(i&IE8d-!6FJoUG!z%jBJkR8s>WJXA6+P&jR~aTDf{S7q?YnXN3_gn_ z>~s3ibt~Cgvn|$Yhg*M_V{(7!DP_(5f%y?Hg*{LlsE-lsDgNZX5Fk&h@UOrMvtd}_ z8WYN~$)bUuHU$$R@EfaeQ_FvB6?(mpMWAxlAu;`)xBfE~n3YusV1>k9YEU%b{KqN) ze;aJ1Td*5+54A9MiYN6wwA{3*@7S?pAV~Ob(i@;(0$Bx!!MCIIA!|f4 zK3?Fa*x#tWWbF>q5{b>U4^259 z<-Cr|(W$zjOeLSsLq9#TdZ79eiWOxhAQy48O_SVpfIO|jxx)&rbA`CJd!Y+Z!T67Y zUnszg$dE>{8|`yLHeah`CEap;*Us~dM?+7-S%E{n*yp<|y!X|bZ(==cQD9sd4zdD* z_@)u}K-31Z!kG}sP+G2mj) z$Zy7gu$*;>dV@pW!q8>(jnapmVH{9a; z&>!vDut#;HS)13f91>|UJC;1G(G)W%=Ak~`oP#v!R_(kXYGlfcvr5mhDXp%>jAOhC z9Laq*K(0RF-@+Aw$N+>ZwB`r6!imVhFSKChVd2?xSu3YBkxD<3I``50c81~Tz*nD^ zt(03Wj$|sxY_O~H1)R(Tb;wRE1Xun ztUTl2i_*&-->QUpudugHk&aRi2mno_&|%TV>&?|8B%26RN7o?=s^pNEE6?7~s~F7L zC?qNdP-Y<6lZa6`8!@;|;Fl*I8+#`KwGZ8mMU~8ypv3gBl3+io64twV=K*QSPM`VtFW>*x;mlj`gw}`UUQ& zQS_o_qhSTJV;RF*P2Ph-k%w{1KtoxkjYoA|_6WaFma}R?Sg~x5hEdwBO?j$T2`9t7 zmiR8$G~CX5Mf`tmVSr|2c~141tB8u0ot3lIDOYZa{~ukXa|acImtMe>lD*CsfC>qn zeI6qbgXzq?sO9JvFwQjY6g>K8r`CSiW zgD5q#dvDv5bHF1;Da^9DLV|E6aI5ju1>$GaBFSj z+Ipsj_&?|z!nLhOwC2krF^zH8=2Zgb>=qJ571Bcz(D>&P02>4o(BjL*Stan%zq7S+6o@Hbzb z&T@PenNI@b>lMzORM-mFjI|Oj2!5deaB%SdY;LGMUmh{=6$rpJ|A+G`0CP47L2hWK zH2{|j5IHD@+J_1}e~BxZHUeCE(|sMfCnh}9yc*CL0D}r+W$ba#JW&0e>y}P-Xpj$g zcuZJA=nm8_(`^lh)vs|q-g}@oC&u`l%KHcf?Vf#&iY6Pg%47T}4JBkG?T=?1()N+1^R#uJO5u?jg?-Lo3V@Du6}?iYw}63c5W;m_3n zct(Hp^byII?@UHcFq7U041SeIc-AT)F;PPu|CtKR${+-=LSibg5P3&=ra?%E70>(+ zro;p$rhqG@(TY;*+o@?LA0`@_fK)F(!n1^miuc`jKcj&0gCg}`vft|p6e~By4toai z?$^mvZ#jE1+G{N+5@H5LLYWauA|7T_=fs<_)}K<%x*(xnEa|pdP7f1LD8S5&z?2H|#r+)# z3CenG)p!M%Nh$;iu<7#;1)!Z&fHb;z%PUvJBzgno+{>dm_Q64+D_esa*J8OV#og~` zeBM~Z);Ds=?N-1DNo28CtpWlStBAymwtSga37E4*NVH`LZWZt*a!L3b4S<(|lN!*& z-PeAWf2ov|)G@ms58QBFy)PYEr2PJgj?XC24oxdE($(qqEIb>?8@YBfzr`vv zcY{R6Yaa>rW=FT>!?C!M^hZIrSiin8-(|X2^C82v>mB_f>ro1!%HcP|%Bo|JFp_&* zfD9eNxw8tbbA%`(y;!cA09N5MlEc7EgAly`6UT%Z8_-jZ3IB`g(bO(QM8Wk9NX*Ua zm*(~LC$$xvl|lH+gg|Q$5rgnIv%s|K(fykJudH;Shnkg@okN73U4FgH&r+Dc4OLB=ReC^uVkRcFRbinCggJqVsf8vr1%Nj#bH**W~ zk(}$goG4|!y#{Y0F0nT`aumyKM_pYdx$-o~5~ng$?Px+&ZzA`M0Que%&Ye~Wq5=?_ z@EC+9Tt6Ejn6ej`>Ja|t)uV;iHz4l#C?PSisdwj920&o9>Df84`A^DDUp4WYdt`Vd$XGDB$r%>Bg0*xbc{dD^r$_jANQJ-;;9j91jLsEGuMMq z%a8~Qs2s$GQzurd&jfx!2igTh|K}1C&kK zQJf(I|6hy9aX2MJYqByD^KxuxUS(j;ULn!E5abmm=M$9ydgMURFU>+&1~$^2Q3ZEI zeS6tUcfE%XhH9)&P!SqOD>L*DLn&gQ3h)^bi=Ji%R-aZThPBD_WvrU|eFn?b%{PXR z4L6&dG|E|IYh=FOY)tPOrz}sSt7*UCSH`i89CtT}_|b3(*vDt;`en+;cG_E|YX+G6 zIz@UZ#4KrAe)BpT)9$CA7m@ckK%Rc#-(VDir~u3;qyUUU!dWrU1^AB^fj^o66ih4# ziMNnX_=T`rpvG2Bb=r@pUsx49wQS_q0Q13r2{3O@Rz?i;0U;>sJjXnr>p@aNFgv^O z+F}uKdAtmNc}xc`kC!c>ra(|2DQ1odgfJ0Fw{W6rB&l=o>ow3d1V3mn70~kfT=#|8 zqc)PTIDtyUFNH8^a43YOV>t%2bOER1#&#-jH5%!Tg+z ze-KTK^ntpZ*QT%1>H~Xa4R4^I)eow~@xOoMRE!O0B7AKP=L`duHKa9X(f1UFBl~XVj|&>nJEL?zjF6 zkV3d<%zEN@U|M>}dg9fe`7Lm?0kDekq2Y2x*ZorxgF~KAoSE0f9!5uRlmk(k8#w0V8^ z=j;>yA~3WN>8bz6FF$M(P85#bFFe8XzO$p_t}jCEEFG^#vCW!^qEY&2D+PIZ7)0_h zRqqoct&Fdcl$Fh`>T%qtATrqEm&vdC?h z;WnFlo1+XX)wk5*j)pU!&YW8@*39=I;qV@vlU=&f$1UtOOP;tGeVNUtxGm3NeKx06 z4=c~u_oJC9j!#tryyI=H!}v4J&Z~k`LiE=6fKm3UZJcE7pjX*O1nQg$67#*ZWM0K! z&ORa0)-fnF&>BL-C;ZJQ0I5OUq#8)MzUl=m0>CQ=&~m&G7{k1BYF-!K4vEIj+h5)! z@tVs!P+L-7U9x3Sui*MaM-)+gs+o*Uyd7GoB$eH?Rq_$bjkl|MFkBXQa#^$f&L`@t z(yJv4KI}g2+rU*y_0nqKq&lKvbWuH{`Cw`y3r381Hk-Hj@p;I__NTD_U z9FY!!6w(n^0wkw_=}CbJr||NI{~A&lc1{J6+yn%mHM?P61%T9Ma8^#?FM|Si8vk4X ze%cIv{CYLgyVA$t1sMJXna|0porSsv(H#@uQhc!hYX)D?vhA#e7dsj;kE^Ba&9ij9 zGYu`!cFmcV{;3jGN_yiFONBXckF#D}=v$zq=2q7wn>8AyEH>`YbtB_TA}v!6mFWF+ zu4)~OLTEymqo7>4e48b?UlEb7PdIl(p>@vv;AN_&|3HYse`X^CzwinF^UYvtlM13B zOBIPJk9aq)GB9VKkf;p6l>#wDe8S(%0dQd8Hr)afiaQ@#l5B^*6|$*WLW{r*fFNk| z<8&wjzd&4~Wca)g#q;XXfQcI8RlN&qDVb^avpf{BGA&GXQhzUg{9~h%algh_X@)$R z;`(9Lb3&J2p+93kXh+AJw_7D|4ZUSsv%@?WWgBiOaG&>vR-*o?1hjUh>KUWm;e3Wi z_4jG)A$K7lPpk0nFbb_N3UPFMn;(N1g%g1RlJmfHA^5-3K3bTiinucZgdo+^n`J&X zgQVtxSviG(PdIZ600jm-TmD=KChiH1DO5~a_RHRqWaa=ILhuh@6b1zb1qDU01PAdE zP=%{jajZxMfI@1Im8`pA4^VKw-nSqx`Gn;x5+W8RrI|MBIfhM9$D(ONYPdW`&AGZ< z(#K;scx5HiwE(|dF|#hXm}aD}(MZ=FyNds-uGXeE=Pr50%Z+7R3TWm3pHuiBIfZj4 z6t)6F;ZZ!Hu;};Wz_gn}sWKMaYIfQ(756ZQvYB1vlT;!x*C2cY(UG zdND%Xn7^eP+xn9Ba;S^ErI&KofubSE9;hL75IcT6qa6b82)B>K|7J&6F`|mVR;nQ} z-o3H&ssg0^fmyYV5{(VP#R92BjKbfH1Jh6>du5$>^_oH~k~4Vj2m~Oxi!U9;D}z^H z;Hg{ED+Gow=V^>5_v((t1u{l%%;hi*@K(LQJVC*!50_!|bV0NBFh_%no?-ZN>7MN} z@eOwc9aW>$KZdP{e#z>}J$8`xWZ*g$stBKSe!UwJRT2H!I@vED#Ac(({hEM0y~4i% zDFntK3@KCxkiyo9;LvQx0X(EoVzFeh5YvW8ZRc@n%T+vI*!^T26&dXg$?eWymlL(D z3SYju@kkcy)}Cf2yT@j|V;R&U%gZWOSP1Xl&35uxh6_NIa`y%@m$W~G?h1qZ@43>j zeGH6I5LDPxY{PTH={r?_MG5!jkeKx(8HKn?HAHKuIug_IwD3O@f>{}bL>Yx}VxWV_ zI6(LdVpic|mQ7aBM{RSp$d-g%RvR=wFLcn51G>1A3`G{5<&X@=1hAB3DmC zKk#Y#BB>pPih4^UxJ49APGz>Mbw-i$3(9P@k5*t4xiIValqyQX&ppqSgre8aBF z>WExLGg3lfTddXfQ3$p4fXKC_qq$q%=4#UN-`G!c<}~%h+qC8 zm6A^O(l7Ys3GisJXD?JZ!UOZmKSBKR-T4 zO)w9A+qlvy^)9 zl{&&RPXmd`%x|ApC782yNK^^%4FZ3v1k+;#Qtq$bb%RwQVP&}WEP-DR{)OPPEC$0t zv=tmBPf7;qN>{3WoUZApOIG$MhMp}Nb zeOk|2?r@@zb7`+1qkCs*N2Oe%;#*nECeGy?mFy34?%uV1u@UMY&cFVsW1M9FuveZ2 z0`&#F`TlX^^ZMqK+7ZsmGb9=sApB7O(^r4`5OD4G_hp2i_Rx~>#|!{|2S$Rxz(87f zMff;05W5qnBz011QDKygzf7>*@`jU|=X~O2T}3E4X{7^iM07;-35yB8GBYvVZenTn z!uYA_>Ex8Op-=}UC8J^XF1au_-uHUr2|-jbl7%hD_J6Zp-UnLe@!%nNmMZXjNo{!Q;_2bi{mK~wLsN;0;-V5x!ba}H;}5N{chfgt86E8 zc6XM2DD$CnuhJ+}gq8wDU@0(KQ3jR*a#y~Q`pF;mMFWvsqlv_H`Gv90r$;bnv+x)1 z{}QBU+k!uR|6z+@GBZ@@p7ai~38N*T9RMg7=*F>vs{&vZ5=zKdT0%ehH3r4RQs|^u zb9@+tFNe`pKNUVFoZldvY45$9=5idm!Rep+YV1uzHvOkdn@x*dy4$*%(xS6ml9bs! zsLVT9=tWkT$)udns*WEHs}1b+!-?N4TTOEsb(*}t0?5-T{5zaN>l}aHT?p@Bh*QWg zn=gPVDp2ohltPX3#nLr~Q@2zrx{M2h)!-O2v>J4=J0#tvE6r{0%qlI8%-NmF>e5qG zuUz-otadDmT4ag%?xo@p1qF1sdJCc5U<e7Tfm42t&(W0)Lhp2(Rq6s>qOR|Mu8F>lVhcZ= zu`*>q%S7)(LRb(kzR)X>DpFvxk=xc`cazWRWs(fvWE^w__HkJ_+c#$gYmK9Vm|~a< zGi@GKZ^!r@*;=@SyrqDA+ehb4DzqkoB_z2F_(=+iyVbfVdvssA3WmJwPIDsQiQd0V zx%SY9?coXIrp>iz>vX?>EiZFkYvIesl0f|++D9I3Tu~wMsqjGyWgVCWAW9*ldwB>) zVyWD%=p#1yMKGTz8t{o6eqah|Nl8KB1uev#H$Vz9coaozp5^2*e~Hh@8Ldt)x6itUZs=wCVy^Fjsovty>Ibe zl$v6JPb2=(eXqlVSy7!RQfBNNP8s@&-1`T~(=7ZOm_m>lSZM3rcm%=}PP7b=-U&{T z0jBaSW2gH6GZR6@8!ZGDt&PN-t7-brWMEcyA>b6wObe<~b&1s+R?izmVy|7HMBq7WYSU*Y8sU^wKv@+p)(rf-(6z7`};yKwHH!d5(}a9{#d_zMk~Q9~NhelN?oxkz|;;m1{!MxCKkYDn)b zz5HIV)b5^lP0F4WN@m$o`>F&$C<~d~4I~kE1Dqu*pxr=><feC(jNGEgc4GU;JIMFSlY5_mNpl`%JT%1{oH)`YUw*#1p_qv4x zGJ~I4x>y*5*9mjsn$rK#U_95dcl^1zbGB4QPtXBJ)8*+BIk!^p#l~_TIF!R>x?j6l z$)2X0O6t5#BBmJSkx?D|Qqk3Da2HDli-+(xUCPj?Kr-JCkgH4hw^)TBB>=MuR{~by zL~;Pg0sLot(4P@9MU%_K>7t;U$)u4!g;=QQYP%~b1%GE)rln-iOg-%lGvSSl7qk z&TzGqpY+H3E5f7l9qlBazu4kYQc0Rv*zTo+7{CHic$gBfd_FgZbM^{>C=gE#46x4p zbG(Q6OHF{y!XI-1&&lrxu7Rr}n=1$o05GTKWme0E;)HP90QiN!8*bROeVM z-hGqruzJ6^$H!qM)amqAjW*v@u__lmZYh3TG`V{QWO!xx7l?(~cw(UsAQn!(^Aj}# zcmrV~E-=$FoI)(TF<48IShzsOPg(*XOXrVRIEd9jcqZv0F?qo$^C|$OR)w>&41W<8 zSd7G4{ILK`#0aecsO(l!C(ox-kE5&6G`x*C0_Xj4uZXST%zcj;tX4K2(zz^sO|tQ($>^p!Z#Sc!+jaw)ZT?!jIKE>H zbp=Xg9!jHYa8Hsw&Tn!pUM?bRvMnWB?`VR|xnVy8>V_yX9}39VHJm%K5TpiRVxbox z7H%V$1|)T0%8KAm`bcR3uB_;KkI2U_ramm^tO%ZgaxbprP7^Dmb5(%lhkcTtOe<^t zZ4seYk}d-E9_T>GSkb)dz?`i^q77tFFL^=If>L} z)4(Jy2!Q5tX_fJH`o))pUmBw`P8jujDr=7?3g5)`>-8j{QNvX1Ofg1Olq zd76iRhgt{*ewbSLruNQ~*~ozR8ieoe@EC)8Ea^pV%hOkas{(9dI#zw8mW!vIZw!;P zCOzgwnfgJ08l%_`$;Wb^*_RtIh+V#ry=cmk;75Q!9+CmhlJ$v|GIK0z16VIKb5CS zBN7XJSn@s=eYpOLVf51GlHs2Er&;LCok@XyqKgU_ z7W#MibezYW(+ROvyRePi?+VD*I-EPW5EPU{!9fDL^rukaPu~PnW`Uz4EV*$Kv{ouc zILk*JU{we@2nocZMHY$%C+dT!BJGmDiW(QV`fh5NyhFp#dK z{ee}1S!MX>z0qZLy{yb5eP_F)JX&$Dd0x_ux@}Q=_b`5y71Ug+w-mzu5{vB2f2ZDr7r^AC% zQuXj&t~#>l8nS1X-74%CRWZu1j9N?f=?eUXaa z8BwhY(F!N_fyBQFrf&yh4ER~VH-YhUO*YEO8^g&UQHe8jW0=X#DC!?5Bz_@#cLoR& zQM`Y>Q>iK*x;DoCxHdj7aRgusC$EjSoj80_vhA+bS~xs#1B3_q;PAlJo-x5EJS4*d z0s4qLPk|5=mU^7+VaJawM5=2FTMf zoIAU)6<=s70kI1uNPZ3wRRFAWgJwBhm+s>Hj*Ka$;Y@ZRGzrWvyRhP^J_4IzfW(An zUzk?~n6qg}G(H6P4V3({e|)B75KP0LqJ#`(r`Qwo`C3uCrHWx=~_~CR~yo`q-^Y-^IS`c%P>H+4gozF4lkHjJ(#m!NK_Bt5J4T$ z{QzheaQ&qoz$Zq8JmJSHrIn=(AWw*Ypuynzb;G!I6QhA3JQ_eV!J;PS-N{SR{cY~q z2zBi1cx%2oXJ`F>^;hCwKbi+;FE>+4xpyX3 zLC{5X%4-3-E!#h!7=Q@GL*2xQyY}dmb{R@vZnQIzJNfvKlX|vdV$XT+;@4?bGD)9a z$JJeY6P(w>a4x7V=(YU)^v)NGXf{wyeCyGXuOzF9+lLGg1LcND%;WF(=T!qp?FeUO z8U8XeP{8-gSN)fN0egmJ(0PHUN#EX)rOo?`S>Sj_dgHU&yI16wX+~p#n-(NIb)xAmPT8TMX<)1NII1Timg`zfZ-9g3ck7Z`@wFvCzP z6fg`g5XOOtSD#3BU|JbjZ09wUO*Nv6@_D$^yGXZA(GXPmx7Hg7hd0jevCkbU)WD(9 zQdT32H71pMVk%rN&;4h15OsEeJ>fs_o)GRJih(+aggyp_DlB=vx5&1i&_R^A`O$AW zhz83I5uT_`NX+$jujW+<=Ij{~6#}TE9JT+K@B1(R0v{frssJ5CNBj4V@Vf2l?yR=S z>B6R{`!D5EaDskgZUF*?)izLTv5}xr;YbcEc7YKMZ#kpp>K)3v6NG)KiXl0?G((n19K3GQXC-Ov|iU z@LlLTRA1QBaE3Or|27~`)9~+L41qfUV+_>*#t`%iO@H^N|9^xO(G=T|g)N_w=`J`h zKnsZso0c8B?_Z>&{7IVLo}dH<|3V3T|E&^up*A5C{a*$VDXvYfW1X&q zkAWOK$IuUO44=(L2c|fOQ&U8 zf(_4Dg`BJ6%3AdqI9~-1EtBYFl|F9Zbl0g`d7$eXV~ekgQZBdjQIb6*sh5lp1>-;l zn)0&#GZ~njZ+MNUZ%B>2Q$(bkbXMa4ND$!LNe4w*v@Jkt5Pm?S5O;90mGq$aWr$mF zR~391JYsU#s!(CpL6t7k&CRZU-6$o)PUWahH41mOkco9|7VTVsw!+1j1J1fFwxX;Z0t??|TjvR7k*es}#7Wz4dCw~Tn*aAPl9*~s8>ymC|X zfNB0G%=h~-esz|M1eP9)IHOs&ZK=GV_>Q+&t7|tdKX>+jN0MqO9VNG;4c8(oJ(1t z)ZUb4S9oIRTAQ~Wt10Pdxis|y7d^k+g~Zt!j-IvR9KRRQV(MU2wP^Du^NnWj^xktG z=Wg_XLFWvoWDIl#wAeZJ+lL9|tDFjOiZO`%CwQm2pawRA%!dIoG!5rYGHk`4DKbJN zL&jNr4E*p5)7KXN@C$bUmMP>H3T$LfewuxsrtGlbOWS*!pJP9SQC_@s0^9lD7 z$=i0Ft*r&%=ROW+p|9_=?X0XCHSh--GVxsmsx5B^g!!=Pd=;Xa3RSXt3^hFwal?gO zhZ?d}cln<(6uq9)m~t$nQF-%DoL_oK^#xQnXeXD>dg4_q#&O9WozZ=MAC(2Sy<_Uv z7%w{Ct$_|KuS7hIqcC2WBywL4lCNDjca&i(9%V=mp$zFK_JuR(hLgj=%*{akL$hp$ zoh)t(+?Tmu}6 zLm&n-uL+^9kpYRBb^zh-Xy#u`)q>l!ssLx$R=3xc{;lsOC{L{R5}^h`4DrpRZJFVl z6uP6twto9^tIe#U%qJ2XTN`^F zQ8$gnP#TTFNf}*Scv&AU(r{g`gyTJ@lO67gVb;?hK3ZcZL-ccZMZLf0Pk9G%0d}GNO`!y#0?I1?xy& z77P}eB0SHSAu+eA6X#V2NDT$EG7JIDaHbAG(E+UE%-4NZI`CshaJZH+7U~z7c>FU2 zq& z_Bo6$1~-n#z{Y-evN(2exJ(#3k#caRk4U zIfw;SrOV$dvJ|t72=|#urJvK@n&hycS!nnI&T&(>mz9`Xe)rXk{`04(Ox1Dc)3t5l zF@e>}4}y*?BX=DjPrLAM&gVQVUr!m2;&4B}4eklP#b|AfZL#~1JQD7F zSm9LHV4Ayu=m$NlobCsij1|YEI*QY;+Q>AmaP|ALfpUwM#x`jp1{gbIeohTc!j1nRH6Zq{Qv(U+h=O-O1loe$%&Q2@ z**GK`9)gU+fpQ|Lfj=AuPM`d~A1n&NlZ+A%uSQp+#eEP5?CtF@12++-9HU3&Hv}3h znlibC1S!XGG%Y&j*AZ+doB7^nshPqKmRjL>Gj#H^jK-{gbjM-?8OmISn;J%SZZm(C zG-t?m&T!KS=9J|>(450%CdVr)@Ksey>rHrXe0=mJ>U@*{dEXU~t8MtVFoqyLu(oDN z%p*#GG2BTo4Tx*OWG^96+fdqd-+yZx7QQn_+_||KiK+PXbza3_&bA>@F`xxvfb}C{ z8~#oVerh86KJYCOG!cbLz$C-tmTuh{oDM~8ew>qo!Ptkd^gIH?%K1L>#!ji945}k> zoVJXy8|^=2cKUDGXu+1gENr}Fwe>`my=BpR1mda+fbMb2;ht`pT> zhFdn+REq9h->^Jj^vg2!cFn_tR^p3rEAa<&m-t35o{`kCyZ(n2@?+(SbL5~_;wb&| zx(`NAL#@OnUoW(gE+DG7xfy}=--5)XbwqN^r%OO;J2)%LkZ5Lr9?%L}8~^3YU-yGx z#9Q`i?Wz?J#Ss35gJa>jz##Ah(sSdh2*9r`oweG`*B>|fT%~ZKY2nD@9JrC}jOgQ4 zP&w(-vACPPkd8%8xZhYwWvlgKEKSO1AFIV**)FW5Z&_ntJ$&XUZMdX?^o7nflGY=O zN*2FmbfC1-A8aT;{J_uH@6rPDo(0I)G@Luluodxz>HPlE7st940p)Jpe;ZIck z6(zqvF2wn7L9{*w8sLMyJg*usXQvQoz)Xh#ZW>rm%pm})f_x%h2?VEb;-Ux-F&pFW zck%H{g&sZ{@GwEYfG5-+pDLCCkH#pyHe0UHag*Vk6{fPOZfXxBp39dlAGWIBZwRUe zoEw|Wo~Do)NKOaa~|c)I|wOE%cuOVHbT1qW;bEA+y~qiK7$DODvxJ!JKYwJE;KmOGdG zTe`8WFWIvjS`TtUXGIe>!Ok3zD2@h+Vo*b7dE#kW&syfUBojpKkGCKO(zYTooh4QC zDg~s*fq!onLZN{I#hFq-4Yr1)&J2D!Dhlm8N{@g0X=wcY@Xr$Pn`EF3-u&m+tF@oy zPyf!TY!p$FDb#poIJ7nK1>cfTNLF z$UdsVld(MZtSApYp;e;>RvO=Yy)IPvq(-!hWw6`I)i{dwnJ+h2%9hA+#bu@TX?hq| z#@+L4j6&zfQ-2jA^O=A=&BDJyGX#+Vh-O$|3}}WEWdo#CK$Z4aW?{Jhe`^+c4Q@rC z?rcM1vcgCHGZmPXSxA&+2$v1q{TC_#&jXWC(+vTxarNoC;ev-^(e?mo8n=!~O?Vnleyl2?6CD@e`ejXWzWV5rAnD;{fMfwq5d_cTRa<-SD1 zh6Hy|HuOAehsNjvr9Ey-yCbpg8m@X(7ixlXX}Z~CcuW z&K+iGog>7x-3wh18ccE!nDIdn(cZH;yV;5N!~&&&){N4^DHnztlCM?z1n+y9?F5R4 z3QQ|Ei7oGGCJ`6dNr(%)gf9$j^DpG=P36x#`0(R8 zt$(vp7N9jCEP2l zFdNBxJxH!r;oo8zg4_VaEo_BYh7-8~(gVTN6T_VmUPVV?HQ?;;h`cOPxjKw$BX4(K z+h>}xe8HWz_qIo+wDF2b;Eskf2(eejT-Q1IHV(Or#|>wYu-xJ?T}zMcu$Qsi%f}3g z$1C;A+WEip?B`H$VTf3dFh6zQt{-^8l}9p6oO*scqTnTvgYKU6 zd6k1Xdxb>h016Hi)cp%NnA%GE=E{be4G$s8IW!93yGQ}%9IhFNAXJrlL#3oXw6-gq zd<`_N>uop4(^5_;-INxQ$zUODA!nb}{Verd%8t0cC~hvdeZxJt7xh`%yNqrtvr?IJ zn7SIY`J}Q}IqF441p0{CZG5%qoprX!uslk!bd<_UiZbmtQJu=jD_D{@M?!OeABCPJYT9%5_b<=vKKMe*Zf!pih4G z|Cn0#{_!Y#9!aus;mhraJGmA}OwE(hd42y$?FVON7ycqEK#xR)6R`__HxB&NIFR+} zTWuDr6m(a_zm&i*>xG&IywT_<;OG!MP>z^p$ZB&1E#?%8IgB%uE&O}BqgphZrCD*A z2uApealF}J20FP{+f`T7P}Jl=`s1AXRb`sZ>lhSd`syzkJ>Wg0g^t4>)ytBPp)1^C zlV9Bwb`W(VnzfCG+*``Y*Djnp#t=jXV2q(Rz!;t&`C&ka4Nch%Of4Wwtmgmuys)4? zh!dcTbm^8KC~624n|fY24+UfuxfTfQCm;nED;noj3g+w@5|skTHT0|?Dg_`X-Y7Y)ggO6iF>Af5ZDdZ+PT-G{8(ISX%e^j0gO zqQg(GIxgsuGN4tT&@Ln(xgF81P424!^7IY=4#yDW24Ie1YwaBh3T_IDi~GSu0RKs6 z?Hx)&pm4%9oLNKE-6O)HS}Y8f1iIzwuKTh=u$qJANxOBVZ@lw-vXq%y=JlqGizn*6 ze8F+i(X$_b)r3xR0r?A&eH+8Y>bd35*Ilh5dGH4bz@Cx*Uddoo+N);?^a;szwc zSzQ?7K3O1I@9jWhg7SLjRS8H91+y{@0m)E=f`WD#?IH>aie+@vXN8t6p{77kASq^g zhQG`e5_I6Fks$xmsamLnh!vEL$$)=$oCO~U6Qeq>EPaA6p6jg%{rbp`Jqf7*@Zt7g}*rM?tvn>6u+`&s^*l{Nf#LiteQ>evQ zVQqCsWtG+qzYHxA>eu-eu+YCkT61d*(+5LxR|4`i4gUhj5H2FKCL9#}lqsB0fSJ?4 z(S|In^J}$%g>K7Semf1sDC9ZKqK9zeq5uXN{z)Ba`@J2Afry<*%=;Lh74x|#oU>o} ziwqHH8_@ojB49*UZVlQ7T0%On_I{N(EC_z+gKlENv)dntf`A$ZE+`M*cpEg2w8h3Q zYdL%JoYz|4b(+k9T;rQ`lg#$1jf~Y&<<9TWo%S1{j8B7eO$7!#PqwCw;}_N z&4%iep(BEy)$2m`>E_&((u|Y2xqa|h2b-|4>9)*lK4!kWYBJvwlBZ?(S3rjNPE!pC zWVo2{RWLnU{A(Q`3Nrk{?kD3&1Z0?eUh)0pDDVSre#{9(3D?0><053ltH!CRL`L6&7M;ii_y@f9#!kI92=m{wqhO zWS4nPLQ)jUOp$pWN(oWMB+5)m#whcYd7esukz`&w=k6fP~2AB(~+=I%BW~R6`m-Cx-;B0shx3exm`dZ?q8sw{@s0 zv&QbMss^msGsKu3f~y7^{)Gm>Uj$-AYxPmXK!LqjO;m7WOf+~9I^mo;+~^x)@DFfk z#MDzaLe0REyZx4X@m4E|7hyvp0(j??HH6eZ$j$n?;&~_!7H51)nDt*zknwmos2=id zCt$bKwB2UumQCMr%+v+>3uREWIk1$4kQm~XJrf7(`Ae8OG0cl z4gVI(5X1){lwmW3GDMRKmi{zY=o$WBswEA&t%Hc419A}1UbCuluwu^;qa47E17R5U zfqys>RGfck*b9{s&MyVxrSO?3Xv~2UxNE@=)JiCBm9lk$hT4WZ%HVG0a6n^ zQ}#^^sutoBaZh+6*OS$nK8SzjGifGhK9HYOaVe3x+*pn?q+@%5`fT!vgDI35dqQ6) zvm{@vl&P8t3Jv@kjN8?N?M-A@=o+q^WvF)p1{YdE;6nao%mWKg4F50Hk|xgSAhMt7 zQdN#szFAcPSZY7GEYt9pp@GrXe=7ich}MEC!m*Buas9%qap+}7oCD85O1N&m^#GD( zzLKy+@6jCJOo-(LS(dgCjnbNcIF=YT^)1H|;}!Hn6KTqc1@{d!rCND$isVlY>`VS~ znc<|PQvcqRPgg~)ra8A4B}b=r`|R)>G&^;4@Qeu};0_DlHVf$s2OX7J!wB?Auw4MK z*D_o=#Sr)cYXsaT&OsEzyd`yD@se<%Wr(4X^nYs^<~-9yJc>F_Rf){~vZ_L`V#^St z5I_ax^>G+1!{1E;KUoEJpD$EHRsne6k`X$v4}g~fprx4U7(P=IIz7Obv7^hkDo$m0 zlBH5ZP&c{nKHhfZ3FUV<@44mttpyuDiu=+FNB7XrWr4TA!NE%&`^gKMb(bH2gapL+}Lva||T_$8hvd4Fjlw zh@A`+64nRf$5tH41$D;^RNOqR!})dU%WeD@22(FmGWg% zCGSbxk}p=SazSn~1b_Yy>`nUQDk?vHtN(uLOc>!FayyM6jSh85)sOp$72h*|+(4|% zdz)8Gy~o5Vp*grPNKWQ;WwOCY4|gEZjKmC&TAqWo;%l$)L;{q^-RH&n4 zAPw-3+`x~}!6LnR(_rzwaA8w3^mJ%kvKJ zV$gwgpnM=e>CbflvJPv%9UB(8Oj2>oJMjYahCuBk9^ivx!%^-Y3D?Q)yj0(S%z6as7aj(e+7mPwcL3zb2-a zqW{I+?NUGPZI2$}47VX1hqZ%5t!`77c9lHe_E6=pdsYpWCkU|9I$Sx+P!A3Z=%Mjt zU^?K+FMsW79V9omWDD;+16IhDoNGLq0?W-8H(|>Se(Y*xY#1 z$J^2Fv2_gqnU29A(;KnB0hx|_=p$U3z{~&QP3x+@{3|vMF&2@*g#!^7OvB$z0Y4QE z2#m=8!^Dh(Y9B%M3~bu8DKOF16EqNjzkx8og#%G?k`p{MZ+SUo_R1U!IumqzYvWck z#e0gip4wi8yKp%HqEOh!S#F)&rw}g9x7OSHh0~77*5MX$OgBxK z_uo&Rj1;6D98SN__C==FyN$>}Q_T)Z(z5w23zkm<*lQTBoMWhW1BMpTK^#N6C1n8P zMqoZjXxC9G^z&lye`^>fj-Ehd7oVi6oJvhxRUKHdVTe%&;JSgnf1v}gVfekBSg44^ z0ksokVlpr>fS#uM?+*gn8g}r|vCJr|d+$!`mp^-;${!h&Bley{FgJO#w6Bu6;5+Mq z`seXcvUXP(bR1hJ1{+=_6e@ej_qy!x&t)25=3bZc#6Y<+B1nkYLTT2ultq%m=Vi$0 z?d{c`m9{zSu{%$UordAx!5DU;Fox$KjNyECV5x~Hfnr#z3UC}tS)AuZ^B2GLA?q3r|3wmTnuog?2VCU|fPbSw9qExmM|vCabBa$Q9$ht{ zs%#C)UsWYov0Vr#hKqNAP;?+56N4-GyNLi&g4&1Z3{hO~$k?p^6{3UtowTwL}cU`UBKKIk-9Z{;AvP5?3-LxBv@_TVqJ1MsK856z_9MP$gG)Pzu1`5BSLUFF}=#zc=HuOj@3j z_W@Hk?%;7&k)1oVCUOU!r{0$^^b+NCE|~a`-kQ|v(!?}ylAF~)sVXv%KH$z4JIRj7 zX<8cds@DzYm1}T3q`4z9B#wV$KhAZeDgT0UB!r>%n)PA!DBaigS){xEpKK)x%G({Gr~(PKhHeMT;W){s%dUTsS8F3H}DP2RFB(*!gu zra`Huo<;(ZeWt3(hTezCwKPrD)6CrpDW=SkI@}_J>kYiPx3(R@_OSpvUBkb?GfYSE z4BY|GaQA#z;D<5-&|C64FGbkQ?5WHb?TZ4Zu%Q3r?z7s$(&}{#eF2@uSpQG>>P6CDj0$)Ncal6 zAK&_%O*xZ}LYOc+m4^zhZ#|i3Fk)~0<=giG$5I;5oo2_4y^n)!Z1a*gnLV zA%g6~N1Fdi1HLoP@thxmnE?H4xe3U|;>-qSAXZozR%O85qK^k19ph(ll5a{UJ4H4C z`bYO{H%HbbpT2UAzWSI(zv^w}6sAyVjkD>4YUbjDArk`BMBJK9y6SfFZuBj9G>oRU zsl6WiMBisw=Lg!Qbsp#nPNXcikQ3!*MrL7qeh6FL!@q?y1PKBdXSfdF40ob-hI*K# z;2)F}tlP7}qf0h8HGwM(`$zx91^Ud7d9Cb33dJ zWPo-*j425`*v8 zMwbe+#=xBsxF~q+0iec#oYW|eAr;si=sFI3p7V^`T)dGZLCjzS?(v)p2GYr!l|qnv zs>?I3#mfZ^vY#?c4^FudD}@^wW}=#E6KcD6oW}YzvMR+Y3TzrC`}f6v2|%47C4GvX zVHqHgAjUnGGil(&y)D`e+rVt z1iWk3Sk=`jk5wwvso+=dtevdl!fQ-$XmOb8bwI!F^D#jRc;kTiQCzUS|P9I>W$olH!MVlkB$WpQ&RKSM7-X$d1Q*k^YN7q z-bOiJ3Y#*`b}2sh{5KiH{OLlXgOVS-#fk?CPj&QP@5rvCsy(74z`E;oM?#T_h0{l8 z&v#4mi(Jc%5COr)RFw~J-d@#Lf2lR$vW!EF(E;>q|F2&C@81GUeS4H{2@a$N*M8@k zQ<0s6WB@ucn4Kx^QZHrq3avkM&dgxYfVSsurGUSo}G>StwB z$Peu!JDu?$t@hG18%>Oj-_XJf5<34#u7vZHzrh9BVi5J&mT`*et-4X=!D2t+Vgx zD05Rp*MUr)GH(rIPvQr)8+hbmsA*2cQ0wOIUPs)&cC%3G8gE=Hm&c<#>NNd!=oR70 zQHI~9{ogCVLZ4t^5!w8Wq5lklMC5}QyxenpD zt*Qd7*fRWu0>IG1f29E5zXT|q#ZLBuKH@S1Zu1%gZYEw(RV*WKwrkfebX95P7di$> z6P678A_Xar%pC%I&nb?d9(-=v9Dg?0m10+G?r{Ar`3g}F93QEW&ml_Nqma&0{sJQ5 z6Iu;U5^=knKTQa!%pN=?oX<>CvuwW}sxxhwTmJ*l8F39hjjW267?h zK>+;~u%Ml|!9zI#AL&s~)D(=*P|afp-x5rbf*vvxBE?FK14O?=aT~rzRiR5;J zc@3hq&()t!rQx%$--YfZEgs5&n5G@DVF;FjNuHD=z9TGjEgZL+_S7?!u{c$vyb{Z! z9R8T@Sm=Fs-BLo3*kz2!&N88@yyVffszR{TD6p&pLyV3glpPrTu`Jb zF(+?fL%-4+Hbq@tMU%;dDgQ9)jK{3fUxZ_PtamjeiI41XeivesULT($f0tE&;lZx) z^W(3Q_eN~Q^*kM3>XI0%;N4t}<<|t*dttb8l3^#R*HjxK8EP+QAMleBo(JLNNbm6k(KbM=xvy$`o#ypj6Cuc^GJFK%lxRW6!t z{5hwl1Gg5;0vt+1J|s>h{xq?Le8u6G23dJ(75kkwgb&e&pPB2_%>z)=G1eD^ODRE4 zmI>m~SD*ylqn}q*307Baogl(a!|-pg z3_)rDVi}G?EW`Q2fu;9?3v>Xo2&^0a&rAfRUriA8HKtURWo-dlxIn~+92GIz2qH#X z2?*CpZCCeVjM*6r zyE9Zv>Am~Oaxo^JSUcd)2y0PqaiHTZPMz=klnGea1~Gpvl;ogDEB<{7d* z%TpiNp_{9h;2C-9uO*+;IymGSOq$FNVGwG z$f-*g^b+ z8iPS4xjplCA^4{r6NnGxOd#c<^Dqf4lc6R_BpeKQ>bS2=x;;!${51O2y+adMQ_C_{kXP8G+(Yn8?pj_?9;G6W9--q^Tc+xH zc4eeR9=0uy(vjBur=Y9$O!IA8-63+u>AAe*605%bL2h)WdI>yX65if5F9h;5hs|%? zYp?ockN1Q^8_5nHY$bGb`alu@KbC`^&Wuu=2*8n1LIP8IWqSJl zgW?OGp3Af_@H7$J8A_x_@Ve`&6eSx;zK_h`^-*Y=-o&J_lBqFeERdB!QJMEsm0%-l z{8x{mfHn??O;db?CkRK#Sjfb-%d$JGg{jUaZcQ6kyuRgYLVj{;;$hWKp*Rg@*uE^p zLZ@)$I75&XfN_Sd0B1N4IRCMM;F|z^J2n0m761aUJ0Ha}O&hN@Vlbq5k9Sex)o@n@ zL6HUFaBioZjfs7OPpBXR{Qh%-_g_S;YKudI5c>TON4@`5XZW*eMTbH^1d+V9AC7)$ zZD_&Y_(r-uPPY+`6rQynHh9D;l(+oWU&9E80C0ue6Sg;*{ zt4SLrONjd6E#c6FYQBuQv`lKn1CQk`iL;%uKQDYDEme*Qr*bIH(<>pe3> zfct5x%8DzatEvPm_6;#A0n}3d=>4B70XzzPDN?G-TTS?Xv3p5~oxb7U zff<57?_qRd*V9MyIbzH*fSw23b|XCWgCi84kR|6iQY3HT{D-SPHo0EtV=X;n_tq)Y zcA!YY%!@!daxDJb^=8u&1!9hy*V1uv{V{|G1R!(?Z9(@Fbat0@kLjG3ILQL zPyx;7?|!2KLGGs!@kpQo>7TrqFf0Q9+KOvkRN}+Nn>)=5 ztSzmF8(0#|;?Bf^oni6>9g-vI(`8WsXQJs)4SIF{6ced2*QTI}$fuq|er>TqNtyT; zgDzsX2w<;exN?pm=qZP4$I!LJOMeh7m;=6-kRf%HdlF(q#VM<#XWhXe!7UsuA=Dwk zt!_mkqfB*HL4uYnQeHBdob11aG@(iRp5-%SRL@gcZzzy;@z z1>mQ`0p%hKdsPh$jU(W1^oFn=Y9syLPl_rWuzr}~ewKx2%R4DsncA4P%YG!|Cj>kS`8JX#)WLaA22W?E7)2dvIn6>;&c~Xj`rlYUo zMsVc3FtDgwE|4ay$Wfo!t8N!7F<$o>&2+%k1_}gYcxJkGSRZs5b1`P!0 zcJgXM$Uub00n{8&4b1_*U=C0cP0c-!Xig1FWs;XooW8*$^Jg} zS7~i7;}ScrX7DE%vbwal+5$mZVrqCpSiVeY&03oW6bW}e) zi6~m<_H7PMa98M55>8{(LHk;s|F-5;I6$}y1PH&t0m4kjV(*W5ODRC@C;RD zO@7^~D!_`ZLX5E?I4%%|p@tAL4dUG}{Kijaq23youHv{}>8aULKr3Mp$M;1LF}bS< z4P~PdW*fl+dxaoO%o3fK?*4_9i)l{cfLvF2;#F`S&@-`9mPMDQ+)&P1jw6IRL{s)g zaL1M;$-DdZ7Bmg0UfeGc%*q_;Zz_=T?CK;%4288$U=gda{IFjk&XhD^Et#&tfrD6| zCdO8?@NZ!ZL2dxT7*atPLo}&C?>hXWBhZhIpe17d&KR8@Kvk3G2TvSCoZXkQ&@lW~ zP5Lt3i=ZfrRon2%(boF009?wWXM>5^sLLYm$(-+tqT0;ph*aR%tJOqoVME)F81yU1 zKbbacpFw%PXYP%HMBgDCmW?^!yr`McNpUbQE|J-4T8Eg7ymIMb=~7mad2NWeV$ThSE(7{aoj2Dkhdy2`^^bHlJoAX zZxc7edoNnqi-*SeNo5On)kAV!;k1o#Hct*Qhp zwIN)VWBAL&z$44WZ~E^f;HUZ_O=mHEP(K7hLkplaYgw&#bi*Dzc;%Mfe@A(mlI0ALx; z|JlEU4*XC<7TJ3@z@Q4zkJ}n-{Vve)?GFEdlG)Dth%=ZB!~OJjtzL@@348n38Ec|S zhu}iO2&jjQZPF(YE*vogg(GkwVeFy0-xLy#H(MZF(k!VehrfnxT}`K8#eN~i9x|v^ zEdDNrkAU9^0URCperc#VHCqy1l}o^6AinGUf-vEhT{l7RNN$xuVO^JM>2a1&d>x+` zWKPNEGOq(4q?HC%vt8Gz%vbxOH0c)2_Ho_dLzB_eE-$I$G;=0FL2?R>yfyX0@fZSLZvx5UGl6KZ`8ys1^h;A9|f?}GWwGL>cNFdEOJgnY@U?H_7D;F`i3iq8G`TtOfI|x$c6lW@NE#Gz)bdkif$KcQXO(9)8~s9CHvWp%urEP>52vr;r*3o)@hU z!vOspU;_IA0bx0l`6An(BR ziLadISTZir-ZsF6Plt5RCZ`8gXdBo%HZX9G6|MV<1*wDX0e zHVW2{t*ZdbY!+sjU1#_W%Pg_%EFwGb993mtNymSt1IzOc?_x+1{SIfoczdv5Px$!z zw}J80>?4SC&H~cJFy9>drQ}jQ)ItPm2~AB+p+Z9U=d~RAc9jExv_X7ZF3OZ#izGeu zEL|vVHb{yi@{Gytl#j8tc!E;aJ_C-a6&@XYa>F5_>`sA`Od?EgGd}0OPMN%! z*f(=)i=Y6@IpXo^o+xfMS08L&6JV!4dmKmW!O3Y>SGoHv|aFj zm1S7^`W&MErWI9X<266#)m#&<*f9iBfXOl3K;nnh?D#R6)lK$=U(Hs9RFU+dy_8GKaWVXvd@mgQ;%Apu#I6`ZLz~jUQ$GDT~A= zz68Apkn5+RR?M-^Io3nf6WMdL=RSKj`%ryzf^2vyr=&=v1KB6`F?M>Hwtv@#kJ&wRfAs^3$^ zPWU-Jo=t^xk%{P7S5)ndx4yRqiLYGky81?-BDwdKST5yz^Q_`J%dcyf$`T=OS|K{W z02#=2PhC|RSZYPMEZ6Xt!GU=G#WH|?+kg2R3kQP9+rlq};eh~sL;&UjgXdrx08DdU zQzSSt1oww-P@WJ+0HGf*m0V$vYGB+$(GwnWll`^I?WA+c4{;8iyiT7HtMcaHvk)!g zQ}Tn_={M5tQi@Z%d|5r;+G!na72ST0OpJXjq*%r%xg*(%n=uFXL00-d#bj6>8o*x9 z@GsyD(aok1&QRf}yb!9XbpERV9tM8x8G1YR|EHc|_ZKU~Xq7cpW$0+ps;a<>JwuEt z0JjY||MF%3n2}QeG}e$ zO@=8Q_n~JIlM^u{roODL+K{ZK{(LKGka_TxN;{d4GM@|;-6gQSc?5eM!i}jQfWHcU7zln~8LHK3_v&*uGtN+yqOT4Arz}I)Yc`00w?GD7R=i(T z8CbDjh%rM1cMf=B@C$z@1Mp|T_w9hb1qEC(w2fgaL^5V#0{?*jEm=bY#W_ZR<$z-U z$qWLCz=5FNKoO=2w!JJ|TP{i+km-s}yxfh;vd2)qnskTSb?)n$O+8JwGC0z?1~eLM zB>W>|k2&t0=x9hw8Hv3>KY+)*p4;Cbrm8r=DwsK87~5$D*y$Di4U!@F@P|o;2LQ?N z+sQFz5kRj97n~eJ@gY-6P&yFLwn;eH%XS)HXB*=THB?njVq(sxpF+*ILkA24JDV)i z$TbJs_hiPNsN<59x!q6xwqBfosGzfqvk2T7jp=DdH-7HS0J&&7LB4@*^pNH76^Jz?^=W2Yc zq7DG@qJy*7CfuKf3|(~<@R0X8oWt7fi|wBU*l8B79AO9|18alUM^t$LgrViUQ-E0p z7H0+^8F=N3$_#MzhW?xx_%;jtL1uvCUuOn#Z4ue7c2t!SaWSi^0xR|mF=mG#&(I|R zgJ<}=Szx}4P-V@2)ehs2C-=z_qaWylU^9sWv>re`0jjDeR8>_S>MX7GukO{k7PK?` zv}CP&vh*prSmoKPt}dkll1$}9a$DuXL&`(4(q;PdQsw1d5{qy|51$mPDo-t^vIv@0L>lcaI@Sfvl_|H$hz;2rWTTR2ig)juc0T^M30}zJ7zuyxc zApC`C$l+l8?aI*U^9Avd8w8pWXTuCl>??h)ShpFJ$v&}6-L?7k6szZBuUKC7G>^j{ zUGF@1Fj6ug)yS8v(?t?-IABb zIj(kyNA*AprhV?Nsurx+G{mR{8-NycHmzI>3@ymE9oz;z;8!1?s~d>PeXm|6r1}0N zI(a5tRtfdF92sH~GN3`DaW{P*=p48s8NN}+Jl|$uxIRf*Ib4^a$a#8VfVSRZ|KR1^ zJ=E&l#s<%-bOY{e<72%jc|fi#nibc>%x7Q7Y4%sD57Gv=&?bsvdw>87EyI-)3_Ib7 zz_-8v<}m=39Qtk)ChxhWt9YH3k`<~X&M3T-#g$C-IVg^x-mK|^dxstAF!!4s=Kc0n zu8N0+4a#<$8-3rp+m z5cRS4RF$tLpRVegztm!ISw10%3oM&Rg03Z7{Oy1q4&b%mL!z~+(3e15FV}~Alu&30 z%`vAslh@@5z6vIgDe~c*I!x{}pD%d&Z>|-*>3*Z?Re*(zMb(iUA;NWppU6DO7`J=( zpG$U9$Rc$nt&YvGOd0n*MRH0s<3sAKmy{Tdaq{M1I>isX9LU0hR|t3A$h{uixaGd2 zZqy9t07Jwj1p*;#?;XKjqwp{A3x7bB{ZvObuLg^^0hsuOC~E)%zc51Zmi02S!}H$( zZ~#zLYIM-369?wO^N0vOj0%+0HlcJn_C{ z+&*XXX`A%sB(=C|iR)L>1ru6B201erOs$ZV2Dd85s{qbi!fltYUC)rIZG3N_E2)RE z4=34i_6MqtX+umzTQ6cep#(b}!oS2XMAeykzt|pii2(Hpu(S;P${|!A`EMP;ApgTTLc2%)IJBX0~2;j(l4>Z{9zt7ZO(W6)yr@EW%?zW<>cg~8Jo{7Qo3^Dc^ge#{P z>Y=U@cS7{Soy*aI1?IrQlfsTlp+&dF4OeMHZ-Evvic@)wJ{%v1sSYdO?gJ%cM<%|P zkiqQ2Y=~V5myjJUrtM?VZssr=yw-nt;7`VZ@q7n_i?<_H4lo_8i=s00W(4K^}UM!CfW2Z^@H|T{RCjik4vsD4T zaK2+;>3Lv*4*Xyeh8X|1CZVghBO>4>(1F(}ng5v%EXyRs$S#CC27)n|gv*)-zJC{F zIfp;r$N@NI+)Sk6_~zpHw}CPE?YLOqtJis>|JJ5V4fil+5oM`MPq!Xr$*E~d4^U4z zDx z?u;qC(YX$b_qbVS6bI8jwT3Vgh(@BfPJL5xu$xLV-lzj*)QsE`=ZC5UG} zd~)j4nLLDevb;sU^#D?+aQ=g!WIuFLEM6J+nnzf!|G*If3mGE?Tk$hev(c_Bhcq=_ zsW{8~itHt65>)-75W`_((d48uC!55lD3<%VL2zqmb9p~intaeOyQ5)$REkOL=13%N zwL|ii9%^ouoG|T(^K24Z(<`z3a{yc2!M{Z<1g|~SZ8wEZh*~(G6!?V{pjU(o?LqAj z*E|%ojMl#=;U;_8&LBK7rh(zEFY_g&VF_JK9Gu9-C8RMFTF;3(1`Q`%RruPT6ARgt z$rp7q&oLU)2-*>c@K>s?rw}+P&~q*Kbc}@4^NxF`UeVuLGCiD_;)KZl0JI?C>#J4O zf)(3?7_|V33OtI&U=RM`EO6i13%w(BE31BS4{Qd#_i{72qoP94P;j85Z*;r(dpSuH z!HD%O6apgWzVr~Ah$fd`F5P3!W_=CmAQh;5c7Ih6lHAa&nonaWy{)lN<~8Y8P7U95 z+Z0CyU)sH-r`j`O)cOvbl4=cp8a+fQfcwDW=q6qEIqZ%KV4*#@a%>^U2f)}uH-Igi zXPN&(0e&0?pb}DmEmT35kg7WvA7@OMFCk@MLzR$fno58Y(oCp?loQx~=7IKgTzj_Z zPNYU0K9!qJTRgZgpQ(vn*{0$^`Ux@3q|XU%Vz0i{3x~ zM15;XR$0ymC&Z(Y^Hi0t10PmZ0G3(~F3Tmvm=A!O$veF<3IHLMOF4!q0kcnN@nH~9 ziSPqCb@Fe?hwJ78bTMC><@``^2{HcW3$>_Rzu95$eyO(yCS}K>f=E)twEB`aN3Aoz zlFQLpEbc`^|IYdp@&O18v=>YaHrnl<+7wE6-7lj{%}HFu-&0gz?F033aY7@qQ%EbH zt=vR8p4}`ghm8*AV>_q_dws%{Qwu>(0HzkY0BRw+lN{&({38$Wqr9K8LqCcE)H>Gi zOg;UKHkspEr_FW@RXjhh2uG8t?LdB54j_vF$&_~Cig4pMP64zcJOY;uJ<%Oa@x+6I zgMe5B(#C%lA1*C9kEr)|rmB3?bcJ;_H-sxT41pM6Y8Y5Yg()}q%Mv2A7QpG@?>7V% zBDyj%GW@)}ywKm!^A9yO4GlFlH97b>{CwIVP2kj`QF=7+eyW~Cf_y{RiMTT`+kCr_ z+J(e!g1#6&dCCfdjz`9>cjvB;OXCbBH2eWUj| z2;$Nc68q%mFr84x_Vz*Sv<&|ex-cDuF6?>w=pXcg;Ku@@`QQLr1{OGm3( zA!$WFyPt~E<#FWdzmq%?d4cxS zaC)zblQMnyG#-ru??+RH`azjj0nwM+t`(Bv5`e6Lx*TVSQHTzEoou@d2e;i0-cXY3 zS5IG_#`aGF?6nJ5PA&wk zz5(16&yx#r)88mnWIOtZKmg-JWi-IpSI3c(zu<(SCfE=jpLC6(iG|LH`0r>cG!-zX z9{Ym@VRySTV)VuZs>+nif>nM0m%1WamQjeYa}4elz(gzhodCeuK`;l*3&61Wx2H4# zJSEUC2?LgJd;ubDL1%BuvU)VfK(i`#f7195+d35~71%%5qPyx?t zjsKYnEXyXuNG*hm1zIu01%9Uj2nbvF{cFHV8QcznNx;O!(quO@2?QWE!IJ z9@OQ0a_8hmLdZkJ(zj=vS39KoH8GX-W9q$xSqZnTU*)CPq)TV8{cZp|O~RF93-vN8 z_IGXBv=71-?pw}hz=EK_g1}G-hl6RFf4$^*Nu0h6&;1WBFKkKoMV?JU9uhV!5hu3O zI-Al#UpB}-Iiu!Ic12fzRD3X1)L#hm!6Or5J=UO8X<5#H?5ShGd(g;D?m5sXmESL ztH9Sak@(Ws-YbHwKH=X27J|3{1T5r-=!9q*!7n}tFxZ5A-&uvs4krK0D?%Z_3#Uc# z!pQ(Hd@~7tP~}BLe3;8xs>;d{SGLvM46fKG#Ha;uu|Uti&;o|FYsjHtK#$Btb+%$~ zAU82FaVs-)HGt>k2|xG*3!i}kWBy2PzsOXDxJ=TPwF1WHB*r;&eN3EF6ct|4zTe#X zdLSTZE`CeWsI02e*p(nxB%dKmFXN6KJ05C0*O*H;O>cY=nmVoH&=j18(?V7K_Q*Zf z9FgFd2Sc3J&(%8AG%x;tvrsa$B%^A|fqlP}iLR~g1qX&T>1);m8y)F-`*e3nT|}33 zI!~F~5o1r%jM@#f=VGXhLIinun*@0ZP4ume^6GK6*($8vO!Rh*H|5ZNrVH_x`BnCJ zL?lSGM|8Qx@{|$t7<=N_)|NFZtl0+_g{~Y}*ogubT0y`1gHQVN(IOv6+qx80tG;1_dqg`@hWmvWnihz;IiDpU&aMoGBCJ>%NPmB z7og1BmEeXk&@vjC3)PJVvqDn=$PHD)T#^H?=ID#}r>pqv-?sO~rYONz*L2*9x^t_J zmL^ds@8xPO+*kN+v|&7@#;&4{;i2%uFEg@JLY>E-%rQ|`Xh&K)JP8%h45z!r-)dQC zc(Rlyu_mu zcVgSHaNy#ap`<^r8OmH{&Cp^Y$N?IFc$9Mq2!T&U&Z-K*ik(7?LIB4GFyV<{UO^lN zh4^k2l0#S$OR7g*9sa$I=5r}2;W=R2;9pPyd_f|<+9UwLh1&wG0$-=zECkZALj zmW$^CEI)UjtWM*L<0(G>Ir78%8r729w`SSary15N)N*JaIjzx>dFOej)|DvpVe1F4 z-!!Y(7-)eBY_?5>A(}rQ6?7fo8P>@638i@2 zG@3}xw=h&p|3mZGZTPsz7s3=NL&rtYEgZKX%RnAj2A|+evFgWbKV*s*D?%v{14O-x zD^P@YIn}GG2uqCy%Q6iCxp1**2=UD8NfsX)|E_=p92=8A27LqibdS1%d3)o)JVa_oAs)Y=lsxU z`FK%HR=`6MrB4DpokIau%(eaxc6W7OmAB&LdXUpS=xKJ$_R|o3v)EZnBWFXalV3VV zs~V!-_y>h-6Jx>hZv*Ug4gUhSFdfA$bO+qRYv|Dca|6jQMuUVau9Wq`c^{-YS_YD= z1J92dQz4(eZ%w${?(ShEoKCkRHdimvGxV^F;4sGQKm#f}K#9%{@IcuCr6F=SJ78JU zF^1-q-H82xSGHIPkU#?vowbkQA7 zW90YDhMHQ;&t;T9-^iEWHf&|_sBDw+6XCe=Q%_#uGKQVk{a{{JNAKh`=}#BUUubFF z`}tN-P*1`*mX8J4X&wFzXrbP96tr+1fELc58ZW5=3j#z?DY=Rzs+3$rouvJzQu6Hj ze^E;Qc6p`bi&dZ-2moR<+6}0{O`qxiOa+$b9%AAa{t{32yLkX)3CSM=eyJri4{+7o zI5tW2Z7)!w5AO{jj3L}UQq}XiPe3>_FO@UW-fD8!Mhd!Cx?KXcpYNP-E*sd@XRF<2 z)%)lp^L?c@0jp1?MI4_jg!4X(mL##!axpw~81FSz`x?i-PyMuEb8Sv=pUtKA%}$1s zH|U!9Pg^!=1O?>X#P$ya>~s)UjxE&tflmfL6+oh>^XEoVjM%~*_&Q{TF0sk=@(zXz z%)@ssFE*0u^1eBca>KsSHG+mh>ulNr^H3CR9`3Up(CHlV3Dr5PuX(m>nlp&rh1J_;6I z8Sju0Cx^#V(<@trv~si zAV#$IEP5O$X!{xsKhWuMI4AYD77`pL_kfFW#`e4ocAh3DhFb)4oKN(X4u=*Ag)?e9 zo&E4EhU<&!N3q@fSI2hOCjH|dEG#?5`|Qq!x;urHW88H&)@5BOWFNa%>a`>8-FdUo zJLPfv-s$XfzUO!vZ)l>0IP~CS+U-+ov3p;Lg_njaXBX<-kd}1a!ifj53-Oi|0nFwh zLK`1yBfVXilZ6TsG0A4B`ja}oR>-TldLuXC4GYoXqhkTYl+t)|)#yHI;cCHTdTaNa%)T3kT%KeZ1NfeIk9pF>h` z`$5a9O2LZlLyS@Y*ACpp;1d3B9GL$&&;#SZEMraDv5au&-48YeHADl7D9;d%7Qnwi zUk9Go99TS$;M&R!g|*zsIexOeGrmecDrsYdh|+ZQ#wqUZch@XPizUW5nwsS0<(|*U8>^=!!ZWhiazhh;&veb(sZsfev=!fz+KFZ@S zhz=Wmd|ls?nVr&n>$X|8rt#|@CAR{KPmy%Zbb@pb=+MZSc^I7m>ZMo1uAY zd^hzF$pxY=qcblFZql=!=6xe&m~M}B4W+Twx~;-ki-BfJ4Xq9F3cArW)2>?R=77UV zGEFqP!D-U_7u95o1)&sZ0iu2k5`+%{-t4QnHe9iL2!chJV*~+oe@Ziey~A%TQ{nco zGW2-?^jV;lgNhgex~XYnsv?*SkWjqvpd$Oa1g%V=)A^^r46PrMrt=qwI4Ah#bjeiE zYqMGRnf7suigFS8@)Vr#y_$a7d$onmwx4_R)V!jIh(tvBmaxBr%^StihL}Uu(C)6T*Sa1FE}^SBxr{T!3D4SU>9t-dz}Lq*>cR>tiXWwqqa3 zoMzd@eI(`4=f_+HLy!pY6?8wo^*NhzCcUc)5&>qX@=(F`ttaygM(nM>K>Y&@hx{Y> zIY>ad70j6aX>mdl|^U-Tu5)m4Ov|h#1SsAP=!VW$mBp0K6|;Fb$k)jNB&xGR5-`Li{p3 z2?Xy0P;)RF_|6HPzoc>}AOGMU<06AQCkpiM8$RqCf7Iojh@?S zzF`iAHisjl{Us-#1+ys=bLke|D|GtMaCW?_$!fB<(BM$$+?4NJ>VFgUQ!L#LqoxQ89{^ss3~mrPzD=A@zHgToB3j^c)m>U z1Z3l{q{sn=V#fL}CrS?)MH?R7D{N3MDR?HQYa5QVz<7aVx6tdy0iWxGjRte;uj`H; z<}%~Lw<*(U&sOIvY~m0dpL?Sq(Rb_8X1oYPy&&dP3BLn1VFJ0D@6B)AYwwy~fHPV= z7K{TaK)AF)dJxuBzp8q$Viysk9-was{!|Z8$wItRaC4wPQZd-*OaInFV+SkLTn;ge zs-8YI;ASVM7qsrLhb)C_GqGh!vOF4zvO;gqlzzQxbH!I6YM(!auuZd8&ikA@CtjYk zcvf6j;#uaBw5x)NO-PBk_7nKCpFU6Nvty5Rb=K9_0v8Eq!)Aoes-Jawd+Y6N>9ntJ zLC~dZ%GjPT#zF&eW4-eH57M5JyK#*$LY zFrJ6Uwsm}v-{4JIWSSaEvun^ZE8q7FzD~eGDw-fxgyY=Gu*d+~C*w&rbg9WALt48oQ6Gd)LZYbdh_bsq}saQM(=US7a8 zVkXM_aW1Jx{Ysq=muSUWHghFnWBOVN&U|gHTaMa7Va|soHcFjlrBiCDy=#JhzaHCX z0qpe-SB^6Toup8Pa2X#2^CJKcWEMa;!}+Vj!XWgC(G;mCuayvqi8-JCPqyK;0jj=1 z_P0Y)gME<6JmaD)H9ScBCv zG{(f{(KHm04;iHl`d10%Li0NCjo z{vDuUC#vN13IsGHTJl@qpM~E7un;tOfLp^9q6ErxaBCQBH$0)5FC#k6w!!qR?Qngo zQE4!^F&f%oq_tbE>>$T*)7d2F=o@L*J}7Te+n`lSr?p*QpYnW*;mA5;&B7LPHrsNq zqE2aIt`6GQ^87iP+8HO~MLgvv{p9aowB%}gr~1*J)C_m24Phb>0Yvr_AD{#e!mq8W z5-c?j{CndNN)Wh|;{2%+%twjl^8-;~6c&(gsO8vr4Ei7tzlB!V_~VX#r7x>_PT~EXBu{*E*6C$rlIh^ zkOJ3wk9wjeaBe6Q`=C*JVCz(6-0tLvR}Uo2q6oMm$KrEtTr(|^Att?1=*n`#@FJpU&`+ir-Av9n>B_iQ*ummFS68L(*%ZG^lT@57LY(chB}JB5 z($C*0W|A)@1vx+k5RXEx04a!c%U)F}Sg~b@F-!zU1=80pb`hYHWEbwmmx$0aUiI>m{rW&@Wq|baCG()d^Ge4Y!?b*V*^)y5`NV5ioK^Ar&qqg%>q41dDxcQPbMEbk!6RWPwx4)A7S3k)@uF7mQQw>EUn$y> z73`W6%q)|+z4{*;luW?1ne{q{{r4H9mX5(4n*S(ftk3l%Y1%^ut)%4yrD`zY!#FU zIT$avJapJ^=0!mJEKhX+Y9L7Qj5+KAKKqf^)0#iXZ&KUo=pm4+Qd-LSIa?n3I1p|o zpKl7jes z?si7Utt|Boj~s7tr|aPN;V=Hc^R~`SovXI^!Dz3V+H6+pgTO9ZKPLxcwj?;pQim$tTmd6I6DjZWd{v!h(eGWd6buzgvGt;XTsLL7ouA3_}J zF#*Kkw;TaxIe_Ct2ME{g0dXSP;M530Hvl;Mc6>O}6Z%ZJW@jtOY}mf`vQMOr$H}!# zOCkpAN8`yU2lE=%*HhKrbNL{_c{Pv!Ei$Br=hn{VBRLO@%*rPHqMU~5@-7!o#7y=ynTD4D(~uv%FkEsPSkO}lSC{_WPu8Q1xu2{p11Hzt>LZ!e1rRPikhR;3LheX2 zE~BkLKA;-Ve!DQxe_J$G7*c`EInP}O_FVg+=+8y=UXZ#O4OM|YMahm59zNwDAA$=<*?`+p( zsFNLosQq~NRFvlNj`h#yBrezK7fXJo?p403Q?Bdk{A@>6Yql&q!R5Qy?i^sJiTHOA zhhPgxxdZp;Gr#JWvc`n>cxadZJiR zPt>b%8>%NdthGlK&x8>}mf)Q2r8^w^Y|DMoaFbFfOJF?d6A0ZHP~EZ9cASA^AnW?e z-^f4^Pyj^y03-vQ-Vaw*29_EKmgOJ<%Hd)c09{%B$1;FnO-T3u*gNldD*OMBE0u~k zC83NKDk4#wtUENV#$75SQB=3GvK5($5LsDCqR76D?3I1&Rn|Gi$zBmzzt44@qgy$o zTYi1d<8fbq)P3vk>*amDU$6J;ISIuEL407%woPC(5Qhwdzw)3Dg)8l!8JMI#nQxG? z-|oc{o3Acg^D0WN`Mgi!IwoGjinbdzv_guX4YB=Nx$fL!)Rd|WDC8yWC``3=qugZI zsBu%HZ8?uxxc4B^RD~wup4yJ9gX!n9^a`b)@7&RIHjl^$0^)TL7mqn?f`9ur;Q?oJ zd=<=E6mHZS+BjWNme`OhFvBTOAdAuS=zC621!%`!ptCrlMMR6l#=WJ480$DDWJ-{)mV;N1y@eLsd(v z0gJW{3Fe959s(5UztaG2^23M0rm#n4^bVJ$lm4~}2`+`VciI8?JPG$r06q;)fdfRr zansCV>YJV9G9QJ!NIy^?Q`3`m&=5508_9FwlMxXd8xY((#TAuQ*#A1MbGt@Esm8E5*MY($z7%&&<#0MUO(X}>M4#a_(ZwMX>dtu=gi$~O5NTFUKnI?q zdw!b^EXXkgti#!_I~*ZUm^~vvMnZ`_fqDYGWjGx#p0*67Bq%Azb?_&KHCLeX0&3CK zkKJ(3rc`X-b>%zk&uwc}LW+mYe71qD|GOmbKGpm&dcQUkg@{y)QW4rMfH8+m+QHiw znHX`e`FU}mfIDfJaQr6iiM&S-ytr;>*WJ@$yV9JgnqTGKZSRxxrMKgxHiqWEi$l)@ z+u3NNFcK2#t0T{_g_@a#o@g^4p3Jp;YSwG+p(sQ|edw984)J&&*Hx`?>MYzBP~qwC zU*(BDLLR`AX~ncPQe~~vg?(hlla$ET_LCmTy1#q?MZwsC%B+;sb81|MqZ=(Aci4o( z9j=0KhpR}o!ptPx9pfhGN1Je8318C#vl~fDi_QjbjoF%4E@aW^|@PH!-4=5Ob@BlY8c3vsS04jheDTk!MqR5(KDffhn_6|Xsh>#R4r*k?m zTMFhZG@h9rm~nLICc4@JAsQ?N7)Ma=0K_;dxet(?*zP=%^10=r zU>8k^RY>_f%H4#0a9QP^%qUNDueqtjWOzcCj&9~7vk!7TR)Tf?J29iavfN{2lp81= z83Gyn_)mNLUc6C7d(gVu)u*+K^~i}9>4(dS`}CMtt;1hNJq(7ahaM315YI9IP68Ap z_&!PWjcWkk9DZ!uedqK)wWW9Ox^X&FODFEamnxR*>ukPflvm70MX?C7^p!@yzi2#!J?f*f@%O45g7iw8o+}A9(&lM zGIYn<($aE7;`s67yfYp;dgwU;etP`)U4T7YAt%SXMxUV@DXgj8@+qs1U!e4Bw{UeXao63mLJ6t^TQ0X<>N>}rZ(We0O@B)5ONLU2ET@v7!hZnCO z#)&}AKz@qwN!mOzQ|q0{M}??9-sM4a@~GZOba3ama5Y=(;2*1QYrojLp?8$UI?Jws zxBTRzO}d$~^~3nf!!x+cLuN%?#RD35PHV*Hr|krBLIw*rjj0mFn7Dv1+?n$)2{7RG zNA&eV8j$@Wc1bm0zD41JoWnWO10^X0J^~gr5*(g+c68{@1Kp1$+xEDg%*NjygTImn z^4WW#!{g)kiqTr)j})}TWoip+IO8vAC%36)rZUC}^>ih4>hz{fn2{Z0vKBl9O#z;= z+!|b6EM1q8$B{`|Pqa+1JF%bAFQ$*W*f|%_vS)uOIvPEpc`bhOZJG1Cx5OSFAYSWm z@zg``yC0$+Mm+}9!L(g!Q`F>(;!(AzS{e?C^AbBYi-y~e5P6F^ zak_`UhI$BM1Tf+7Fd!UiP9GyLWGsNh;4!DKDD7U(6yytPWBH?<3dN)JAU^;Kf`l93>OjjB}jPu){(W^cBItQSy`Gr~x#|5sf~XlWQr zLZYHpv#qBvf&kf^g1ye0o^=GtGrn3^)d101sJXq;^NLs)Pp17DHEZijYSy%S?NJo8 zMrK(KyZ=y5XV}_r&bTX!Y9qay>e}TKlq9&t;o_NxU?sTjGqz*dYvhz`HY`dUShOW@8YR%BDZT4)w=Evvz znN&F2gmGXm8bk_ri{y;wkN6A*HT7DWvd9L>d+FRwM?Y+RTIn|AP;|fhaXoSa4Vvrm z2q|%I4T;l8{1wzg5G5clBdg6I1=PdoV`RcJ!EEmUWD8GS$EA$T^#@Sjdk0W=F+iS` znOkk{iNTyq7O<{7+cg@vdiSQ}=~jZ8Kz$G?V(NNGoroz?>Z$Uk!&>#Rf$W0OLQGw#2xN^k|VY<3zq3AXe}2S0N9Zv{ah_^6(jiJS4$~ z2?$USzq>@naSz|Rd=f`!9sGxT=-A@_m3s(!3t54IsOd_tc0(*jY?J)eY$;$KNNc~` zyPQ}aqdv129k=oEi>iDBg*wiX=wt$BuX7+l=Kb*^*0 z|8V|pV&)L(d;!A@paqDMELR{1b|KkIDhP}A59bhs<#Z*7XA1(p$#nKV%o!{KY@@$S zVFeJ)t$^95lmfUyV~)LhfpJipCB;yhH5TEg-PgLWc~|&k&Y{M&CVV5OTY~b!H>B>B z&i8)rjSQ8o4UVi;3dz&+VrLBd9AAm;aU$EnBx7)}wbh%CydiufMpeI{!C``w>=X%dl=35jl$Uh|5PXxCSSQGFMe}=q%L=1EeQaG|0U7(S$FKT#j*T#a|hVvM;@O5FCQPmM-vty1eGKkh&PFpM6 z^+3qF>R9f123A)V%J~P#s_BRy)++qBeUe=32HCJm2qWrMW(yp91A^ALt z*gp!0*Fao6_fQFUzQ_S_4>^893cfcG1%Eg95Cc>I(boh?L6P~;l1jm%4Mc)c02dJy z{0k{SKwV|v3V9iw89K$UwORpyk23?oy@5Cmd&D6}ML#OipB z^Y2-c+GTPaJ2<7yz@_;-H*bH?9`&jVo$3c_Z)>Gw26D=d)zN7hXC-q}1qpigo1}-Z zXjGfj^cm*F35ht4BO`ULe9mw1nII$MqHNs5ZJV;kl(@Hq#AzP>8tS3aE4Iz;7Jo*a z2GqmpAOYc#fZ(3+EA=qECGfXW4+nq(Ak4zufeeK8*mExBo^Za2U_sU)LFyr%VK{sG zU(iG_gL=5vCC}1nLTt6?rT1U+1E8AZ(j~Au^rV2!4K{RflfQFmki-Uy-pS%sV>b9S z0vQIr=ZZb~=QRw>fd(L)CLkejeE(ueghAtBj72bgTzsn@PJ zrlE0l1KGLTf8Ey5zfDIlPy|;IR1nAz{74999UT0-fx%YLO~L{9lR&4)5?>%9 zx=?^P^^O8n-#5!Yu;}G?okuD*i(9xzN&k(kM|?C3>Jmrre4o&`>(OTnf!23PrGYV4f{Y$q21cC1C$*M2l;Pmt815G?s`!% z3hRwlH0$_rk)}|N9>k1!fN~<#$N_AzGmkJV2G_7}yf~ zNI;y<;o_NxAV2^!4=n)mux~n5_%lL4ju!&OeD)aw!EX?P=RgAx5uTnv2udR>mQ)B9 z?I03N59Y2E~dU^QJ6AyWE$jsdU?X28(X)9>T^iC$#)A_bosNXhc{%Q4*H@6xWo z8l>et2vG@>N)T>-XZ}H==Z#@JI?n-XtjK6+jLEbrmQAT29c?PME9dLHeDKV{=#JFD zXdjiFCT(AKR^ye!dnq3>yk~Ib*L!kxrxwwdnG>sv_^YUgAW8sJ4@m&^kOJ=_hTwj* z4E%@zw?zE@BzX=O5fqrt?;?6%rBHjnz28Se`MAl)6FhG!uAk;hs=ReaELfnY!@=im zbD(11<0(g;bw0A|p77^xSK(JNE#$s2ddOz@dGf8wI01Q)`AozpAP0yvETjohK3z+y z35zxnfhPPc(q#59fma0h8(~IsVd8$sIxt!T8cSVVT+V73m3wi*g(ZPoAWH!hnxnO{ z&YfiF-`vHAydK>X(H_~Y!{2f}tKZUPcaRQhlQ-9B0;c+DV z7!qzK=wXa<-|*<~V3Idr80^Zwozd`U;)#Vor+tywiNoq)E&p9068J_4CLwWAy?|~j z+?QVLi9Qz0lWB25&6@fZieihl554M>Lp;0{hj@~%U%8_iUqZT5PjcA?@-`A%+O9vg z-Fx4-Zk zxPE=QhT!`3T>ylRN1@wR3zhDy)qLBmUOf8Pj@q48QQpuHTWJx;C7d)m-YR!)!nA6% zyy=pi*P+f^2PqB;sC$OK?YhwQzH9gJ7y~~8r9s<(N$%sE{s+{4<=kq^u+*HQEHt}= zh`p|mc)i2La}VceB$`hIpzXlSnli<_nI8qSYs%K1%JFfyGPKbcYAikChi@$1lK6>J zQ!hI|EIxugrE8_Y0m0GR-tL;J;U2+&XA;*%>c|9Fl0W}wwacHsf9`0mXCiGuiJWDXY3tgCI}92d}bEZ z6f_#B7{p|EKGD|@R`oWy&=#5!N`6ozmL>5f$DRzENMgS(AWq-#*KiM&aJP$A5cd$@ zS-PNUU^+i=#Ev|j{Nsrgy)8b!)i=Zd4M6mjLPAj46uqQEur$6Q6dx$5{~If+j*%ppZNsK3~%;j zv<$iF+7&U%9sFRd$Dl{mmC(hE?Bf^I>eA6}<8A87Fw6~q(8z5XHZBz(ujQ$&i#^ob zmVPVj1zY`pouvd@%3}W?4h|PjJyd#?bN|I*j5H-BNG=PN)+6#RpeZNdmipcyqcHAI?DM2_gF+%c!!o{GjM*dv>q`|G& zQVX_~GJb;WhLe~h*#&&z9N{8KuaJ`iRsCT%CYg8o$$9UR{@{HBB_KZ{ua~g4W;mX* zcIe{KQ0t?-mP}jv|FaGWtOY=0;gVX1zk+!P0t68AP!nPvPWKSZF%!&|0;q>z+B*F0 zj)6H)0fbWzBn9sA&A&|w7Gxa)+99D?AM_BQ|AiF5Uk5XW0(0D%;kfwpKmbhzU^G}w zLql@_%m(mK@NoyrQ#vLul~|KlwyqUj)`IpmUQx_C;Nk#nfI5-iZXRtv$U=*BXAfH& zR$b>-cPl=8B)(Om=(F|Zy{Y=+?IVfU%f(;rKceYvyTID?lT@hr&)D8NX-*pe#1qCG=`DI&OiAdrA(xUgY>JpF4eQ6PBS6^*)+ zV|YDFR{Jx|c%Ta7O;e3%>}pS|jP)Jaht$T}f}2J%Pvp6974K23Dw=v!khrQ>NSJr@ zl#?mCBVi+@twC)ZEp^(Z&HFL0i+jwHw~g(fe8&+KdZtPzmfYbyL-?a$ z+pB+N5xGY|te)Yo0v>`80R%j3f`Et9A%Z#d06!Ao{KB&KPvUgq@=wgkabE|wlU!tD zUs6OLXBxMBk!aJ)hQB(zXboyhBrxj&oF!rv;YzD&=&|ot)t7nrF0Dz zZ5R?11UN)cN^m4tOBaD6_z$b;(&CrHNc3-PgXh-Zd8UE$3ucrOIU!)%pdYFx^0Ple zovEUe?^Zl)?_*YvG~PBwPPv`Zogtecoc{#Bf{vSZ3VM69c0rCemp}^b`1`@QmUs6* zu=V&Pn6{=rvY}!Ux|n=%DxzN{I_&iu)(?C24xPM-(Q_sCTVx`=dW)0PsUIX=| z^0@4P?`7rVNgt(f*#Q$=cEGp8!QyRmfFiHoNS_q;+~85!A?qXWG~e}mtCS6NGrzYR zj+Jdxi=UiN3Zj4tAku0eDR?{fdP${VzLnvE3`CF}Ak;{(oGy)G_E!U*el+_Z@G}G4 zNB}MkNbmAv zDo#HylXEG5kM#5h6v_BqX1qr>-af$Ba@n!qZWu)kL(~Cf<3f~W{q9ctWF!y`H-<3& zOFBvDt2yny(L1nguFe~2T}6od;)r48~CpEdlk=1HUl*hI{mNK$yKH_=b8Y z&h)-L;Lc-{J~hJBL$gCYF;pS5nCJziD^X!J|1i;cpi2YkE}9*>6frrP1r7ZlC#f2& z^o2rvd9>!$feau4h?19nKnJ|A8B3}Ii}nx+>Hy>+K07`8b^nnL5Xcu>BBCxo!0ig) zcA`v5iBw1xz#}?&*Edt;f$qL+gj}@FRX(NOtb4g%3J+(YtpYrn+1b_v@uAM#YZ!IW zLZ`RVRyWj*YqNzW*c_V8Jg_%M0*m<|iF zl$?dgrGPk1#9zZb1VI9rdx$Aa!!yso2LoUlOdlS6RRa97UMM~gCg>+)_7CPtDQUA<^JOy{;tx}#d8mp zzU?I=tO4JC6MP>in&uwPI6M4~dpPslXbtY%C?Zm#ce$EuXxBqNh>L~>1 zQ(GhbtUCq5eK-mD@hpG=Z$F|h;~9{E`jmksm4HQ?hXnIPa0@{Tff&J$BmgcXnUN?K zPyoFHqIGuARY4YvLAQVk2;$LYfqV+%zG&{OdveMTs3XR%$V5-IZoSHQZc5Chh%SfP z`=s{4A}b1a9r*&C=-loNpZ5rc4!l4Ot_tU9h#e_G>$r2>O`#ltua3@wtE0b_jPp=W z>(?0z41Z=|a1|H#BYN5yan_ao3htrOEBJQtD8xNnHSb5kEZ<<`%(_yWfx_QfS2_SB z0AcpnALu~J>xbM+xh9-X2NvWU0_x%H*&nVU2qv%%T&NCc(WRjq99zp3q@_6(z;(d@ zAFiAz{*EntXawCEWC9Ypl)M1jQv=;$M#{~haW&VYYp;dmyjm7y<4v!lmoZ9D+a@os zeIvVdoouXpy-s)9u&oadUwB=D))Xp$OvW$Cp#MuzA{BG0$(oR<+m^|@#<(dlp=>n~ zPb{+}iN2hSIIY9QQxBCwa{fSXT~7%i4k>?L1xoN#6Z7A#0_H#f5KfJd3i!D_Us4rV zw0lS}N(6ThbpHz#nDs#bbr87bT@?j2g^v)Txtm1y?%mrBMUBA&u8T4QY<P5NAiTsxI)k}y zsN=Zm60|>~lF2?_7Bm>_D+q`fXFg`Qq1n(}_C&+WQ75#HgNEU~O(SPpj@vd12jzQL zcAc-G9{wwIorKRL8k#c)5ko{CDu4m!@0u))&%DP z3Och_1f*eZ%MiCBOv$8V2YW%dy!hq~+(XuWm(p?DC0Xt)^@z%=iP_R#^_0d6OrH$u zu(eL^I~pVnL|V(e!^wwb8A6eD+A+-|X?c&!dDLhp>-l?i(5P%P2`Ww-c|+|3nD=%E zR0oUBXAIl;JgE;qDV-$QO5{-@;2TEpU=)CUGywf*S{kOy zi)XJ0X3)@k1RvXck32-5;OY7E?L*g1g%y3zY4|#3kc}AIN<`O?CpKkrLde5NxR~fs za+Ab5D!f||R*|^lrNr?4W1mg?IFmY04|&)Q&dWcJ0y=<5%L@QvkWt&dq++mW+YpEW z;rM_V{p{7joTL-H7QmZADe`59NzBk~YNNVapqJl?@hb{l>OIatg#RQR1@VDq+mTjs znP9$7F0Ytzjp#0I?~MksEq6VsXoRaq{>19IaP3Xe8>wnF_G2c!=P<(4R%ki6%>AYO zx&-cKy{u#;mC7(J!!8gyy5S%jO3`^>OMzY{>cry>dFLN86L)}sI9heU_rH|1WU>w12OG60R!>S>IP5Oj)s zd`f>r(Et4z15y2&Pc_4}>k{a+km%|sjXIMYBZ-*Cry+7-qkv?vqe(UQ&4WZ^rYu3> zJ`Qy2W4_&EY?S*QPz+dadH!?W3driv+o9{c1mp^O(yd&cZ_f|fNaRK4#OWXY3fv(` z5Wu*@RRDK5eT7U|2WEd9%=8a`H|`Jv6adi|9SCHg)jRUH$-u(=LqfR2IlD=JBm?kB z04@(t(V3l|%;{*S0Nq6J(|_{#aeku9 ziOe>?9Vit}Re4tVq6F!Li@Vw%pol(5K%55R;&F!ng%09_O4BqW!U`~Nkf=n$@LG^# z$&mqEn6T5VijrE9xA`yto|c&D>p9$2HZ2KjY$fC|qMI7& z9awJm*s>UlmpSQKv)16*}lc5FV1^4i$ zXr0kE!iHkT(~lIjx`bGF((*m5^!i(M5M329aSe+-ajY(ki^%5yVzmx`74T3=M^!0A ztqB4iHceL*64rwMaqG|=2m!*W1X6>bPfbgz28*^13911UByjrjFVp}>KV7vPEDNs; zUJ$C^k9(q`_D4Za)yY?X&ZsB3^U-O%5p8GrzguR)!H9rqZBZmPC#WHH?{SvBqzEgy9$vLT{2 zsyQagvv;N3?& z^9=$5@StgANd;iO4dH^E!#T49eJKPC!XJ$T)87M18_iDls0`h)wmfzEF?cL8xuCBB z_=^CvlKk`B1@&YeY%EVNZ2O?5w_T4bRCzc&>7>EkW~uyipE&wnrljK0z7~~qv|WI6 zGanmG5Hso~Dpj6WJ~nyE6{Tyl#bSta%iW%J2K>r8g;kc_y9Idku%VlLW3nzA4G?*H zfOyTr#gh)3a3_m`5b02GUMZOE7QidQ>4Gw6+3$VBD*^O;dla%=HgQyF2GPjQ^bKjg z@eTWxWW8%>G=|N&4uqQUp(NrXWD6^}OV^jPjHLyPi5So}3a+Y#=qg<^Vj=Zg-%-^l4YFO5q(0tA@&X?-0aE{5AJ|6UECz0Kz z$HogrMA|S>H58?G+HGZ}@9a zhfO%t;r>$qbvT_OAn>DL7>KA10TJU#;9UXt+RS|vR_Ge%YuO5-(Mg;3$6Q_gX?gdF zAA;?KQ4Gl1!2D7U_tgF zU>?r44@nT9h=66ma07hYkf3?UZi!Gf1-FI*T@l~_8DCj?9I_Sd8g&eKwfr2eV_3RU z`Hv$_be?F9qg7BrTz=#!Pb#NEP;qM;*WLRQ$&Cah1zMn_Ks7o2s9n)PIfYo`ItER( z+zD<+RzY0@;pR&7dKTlqhWQxOk#$-zvS*w}p8aPY{;spbUqCzzW@F~A%`kZ0^T8Qk-;hCpBzw(fbR{7Zryc9 z;}BovVy+{{BR^8(Mgt`k_W%$n;*MtL1e-%BQX~UMifU#YATP9u0B8WB z@*uodZNxxy^c;Fky)t#gt?081di-UA?E`gvuXE%IY=r$78&{1}jPkT(Jt$W}NhA51 z$IZzlIKvWR&;t(a#$k*J_b~6Qykv)a83ZpRUr4PrseRXK>fW-C<%py>{qB3jJr@wC zbNKof01rWeVAa4%`V3J3JXFG`iRP1mQXm6h82I}{d1wY=z2I6>D}~j@_eXb{_qGUd z3D_&dFpsbEFRDW^Uiu(uP-;)+s*o*7e)Oit^_9WRM~&;`0MlR+Vj4(zh>^2!beVG{ zc0twwUm_+88VAOMapS-a<80SMtKO@`U+p!xGX|kgna(+S^0Uu(Xix?e0MS$h$$-6M z&2N)|1(}BgiHA^zz$|>(ztVvj1mrgiFQC%GKpIg~L_bpLdRwM?J>PIS;?=%e;F(4GPG$6<3SoZq1dsm_&Pf656wLuWM`R* z45FGwon`-OxIGflJm{-&Y*>atoLF6-4o9|anE*|rM(SGYz_XS#fT_j!Xu z7uYI*HWIXRa&uWXa~=6-qd$cjvMw!~PX;hR0T6vJUIQ8E{?faoGO%b1kzkYvvJe5* z@MmQJ&KHZ2FVn(fPZQyL<5T$>`*0V?`{)4L?2`l=`~d1{AKmjJe>#kM&EGX2qBL|O znokX?6-Sb?IeH_tCJO_}Wy9;@wZ_2F5tVPKqM*^>NP$c67m-JU9O;7_xyq6X2lnK2 zF84YTo3Ac=p}OW(^oOhA+4pQL!kiyi-X-={B4TwAe--Qy1PNf+;S93j97DmXncoJU zk}uaktg(^-D?`z~4NIZfOXv}gNDREKlh$(rk*ms`mFRsH+?K;Aiy!{yg~F zOwuV_U%8I24*EdgJMP{9rh+>#4c+pTdjlva;RK{Z?jvfh9vh4pTKUhbx7Kn-oyCY- zxtx=(MAj?CWqTepRyZ$A`$e8f^Hx?7xBkmIuYS#R7nb7cr1yP#IW|ImjwS;AgBgD8 zWcpQ9l$RNjk&{t05ff*X9|Zi@T1pUb@;}Zx{CY(IgU#l6{lE1L=3pI`iikZ`3}6DS z#h#k(^s+a^awIl|Mq##X;oPt3#*-?yvtdwz!j?Aq#!J`DdpoSd-#)s%S~urxz1Asx zlM1^wYSY=U!{mj*4$Z-9k8ny40q=fznctGW`}3^`7vvuj3=rUVen0!}!#l#6vw%15 zL1=5IBHF=b0J=NI%>{p~9{p@+eT;enW5@c=wMamYD)w?$QtgEH-mpnN>MoXA87+~b z01C2;lsXKphKfx`9=o$Bx*95=rCj558x9yWV#K^08HII6Lr1Ebv1TG-RXOS_)TL|I z*7`~LqvsWyc%IDVhHBQ7x@y)`MkortLT`G?8YEAs zrjDvd6SlB#M};T)a41iv?VoDa>k?2DJSTnVH{bN3SB%-WeBH-o8w6gG*sjzu%j)Af zyZNL3H1IU%(T2lkBn?h&Kkl}k>g0;N{Q=t!uTmu;Cn1?yK{B<5mxM&8xLBq?SF-rj z>U<8=a2^*~xxZnp;b^NNp?l*wL}2!QP^&y>x{ahKY1%j(rMBkP5umz$i|>2m(4(CW z{v-gILJLKFrcfB?6HvjQ1gcykXEO7Rn~m4d;>*eMgf-hIsd}u8gpfe+!K>%EWaUD=#ehQo*;yx2_ zK0#!`#nP--QF1Eb>GxbL4%#rHhWJ4vp%Z8%l#1Wq`o&w0yeG^cqOIReO+b;!K&NHG zQ-(IBVMGi2*(HEBhq-FU!4&m(>MWI}a%ybZVd;ub@48snJ6l@1ITLw|h&Y|YUjsXA z!W}N!K(IqHoG#3v1n_d;8>gVuO2(f&kldd(m~Fk_6SbaCr{5kI9$=%m(Wci^@?JH= z1zZ_*?NPn7?@XpRdryT{z+LZ)z*bzBWw z8YTFnWaVF}JQK2(7?r*u1RqFxbgEFJs%~^&mOLgeXU;=MOqJngUvM?0i|XonQICITgV z|0(n%$?t}SZ^MPqu5e}(0RiOUw@n1k8W^?ysyy2@8n}A*reu76z@#Qn|9&+4CTK2* zjD+$7=}>-v2N(uWX=xrPEydY}ubF<(TO0*s0Ff5*253QV9~ZA2ow>1=gs!d z{`p*l^qBzy8E}1!Ulhs(!*&7mM=KVJ4@_qWcEPg&SQVm6cQWW7ak-w~Z||})NCH*m ztr4HvobGFLjLFA98$IeOtKT3z(uW!J(ztwYQ{|M5r^KO78h}F&QTJ4@u8FL96Rj5A z*p!A=bP;n8*p$P>c1!v$>d?ux&Sb<6LMKk=@R#rogJIs`zSDqrINd`4+(hUA>H9YG zZ?^=H4*U%7aP6sHhxpj(o1=NY52kZOjMR+W3Q<&oZ>KEu(F%^Qb0ob*<{Jt!fCeB+ zMj;`HzCZQbgkV9&A>bVndf`D2fgYisFQ;>o{^3xd1cYFcxgbbx`u9ISe~ARJ4fkI( zDlhYDz;O&>TAmGo0`ncWWo&o&Dx0Pm<)(WSS$5lJB9BQp$a<^u*K5Z%n-r&#PC3d8 zYaP)UD*W55llC^Na{(582a}M&=0Qwer)52@@$NB=Dta>Jg9*;gHQ~<7 z|1%DcxTe3#8#Ubh$IvUls+3_SLvbFuBN3!lOA3#ws zcAzpVCH0&dmmMS_2~ks8Jnc~FH5=na^Oy*yNBHl81q#4V(GJbPFZQz6vq&azdAZ;} zBolooKO|qvuG-=8@j?63yG^Cd%t_?EM(2gpFyr$s2+F{_k7&vX0|M}*+wH(oS_F%> z3<(yL!R-Ty%YIq`;635DaI@#jac73F9GwvBA3JGYzm5Z)f&W;&=Zi5)lQ)k^pme9H zR)5<-!8y5!@Wi@LBZrzkIQ6RBlOEw|c0f%a9VvV{FYwh~h`JH^xAF&7wW6@ho0+}^ zlWt`$1_b28aQo&ovw6I1&kkr5_ib54RbME1qa9Z$P_JEtzOiG zIT;nxO(r2Ex8S}07cK2xYHnC&MSew7L%Du`&MCjUsLkGFp-<&4L*>-BbAC`)*?luZ z_r=IE;+_hK@b+-=v_p^`fXRmUA+jO<{CJM|PG z7wrc+4FBl$TBGmDE%8A&>1z6RcBUvaE2+~C0`6gVJ)*-Q=xy_ho{?bMBDP6`O2f*e z(j&>}J)|xS+zQ$hmr`K+(4ixTqI&)|0w4egvld7Nsg2kiJf|U-x55g5C9$jY@~s15kV=D3AjGwRbhD=bGApduJ~eH?S+W; zKFz!x?riL_g0a~1@<93s*#N{<9kMQ+BEV~ z$@ysFWP5J*XoQN?80?{cFnXTYIq1ZDd$@SkA&3yLoLNbavI49_e24jhlmOZgd{a+$ zv;kL7)*?fgby)X`G{#Y9XrsmlDxpBfoNgH_>`w&ea~OKRinV2d0=`+ zfLjNAddau<4MMqh1`Hh^Iu6Y4MG86u8*@0s?>43y=ArZr)j7yTN$$`f$Z0r?zLp)r z75-NPZ}-$2jDp=~PUiO}pGGRs(Z%kZmq}BG_$$4KkbLqK@e~Q0wHYHfM+#ig$e;AK zSlL-~QcP00mB_0KiPJ>u?=|W+Bh6s z%7x*gtwWF~n*FZBl>;oZodM_)(aeoNN5OYFg0JHMOh5;JG=P!dO+%g=xHiOx2;jet zgCqg}^VjDYN;$nY2QjJXqV95T5hORLZIQ|y+vK-hIBs;q;NO{Xmc0u1cHJ8u2^_7* zI-+~cOm`=`C2t$sLAjPPhe3+pS9`zO-%82K(QS7MY8?hGTpvC;#!lop0^)QIe+BKZ z2}e8p1EL+0;8O&+TSignN8h`L1XBdNcE#`dw0wA{If2Sj!5dHsL1Y;yEL4nH4l+e5 z7lg8XFF$lrE)^*Bx7x5tFWpALh?YN5`3lravc)*veKQl(OTr%wB1QVgB;C769ebvl z6bz~ZI=HE^RP%iqL;)2*q zv$he2qj|jW2r7Ap*B6L#bwO~kp63T8Bmt4}#=#Hpbl>#N> zG}MFoNM~n*v&4RIM4X3*zW{lNFFl2jhpc!D5uOi}rpX4=B?PlS3o45KI(%4j8&pNI zHu=x+y8dmP&UpSYxI6TUC;`{UVxXN!vLd2v?Wr7};Val8 z;r=k)?OFk*`l*E#mu3J3K$LVqGLYDqyreR)XbX{GI}zMLU_szF|BrM4ZYKhc0STQM zI>oQG8j2Qz2X}+awfBt^z@K|cvRWeGZv~ifbXY3esiY+s1rX>KXT73XnHhxRM%o6V zi?~|#ZA6$%6UPUUjm-drF2?yNUa&3SGUV(*HT9xe#fa#u`q^()H8LmN#snyb8JLiV zv`o+~`RxiUY_(#<&Nw1g1MycO4?&m!LLOd#kcW7h0q_)|K+zmxu$&tqUV5Ue{gXJI z_zdKsV~hXqMIM#`9Y8e2M*}hNE~s2mF<7*FNKg#mCW735AqMc0;Oo(0L6E?D7;z?= zo9s^k-5kZ^VCdF-3UBYUmt_MGvKYs$huel-e9S;3Xd8$GttVrqRB=dQVDo-aIVI^f zh~%V+ztnQn^8-y!rNIU42^Z{MMO}jr*P@GamiZksp&!)EkE3s-9lsFux2ka_%AsD! z_~C?w^OKJ*#C~!>gzn+unTJZR*jBolZ;U<-n1?m<4g}wR7JS!S`bIGN_coVez}t`L zvyK50FcI6mq!KXSns7nZ;hX`2CVK+a#Xp({X5AhlzTO(9aY46-vJm$SV9=q*)KATT zH?s3wX;x|Ntq8S<#+I~}h-_v%m0Qw!$jatVW;|jj2P7YQ+@4&AlQSU(Ghodw7uJ2D zFubP#a<#^3N*;_-}(5+rs~;=F$Nm0|>K9NDCUCTk$RBws6trAweyGdPvhs|AiL7 z-v%?rfyB3HDUI#$SwRwXdf4s^pCMasxgC!#1H*uX{I#qNT!krZTQl_R&w1-Q2ke%< z%J0)E?`yX*>`A<0+j-Z-15CUIbuAaY+!-4_88l+cox*qMv4R$J9aQ+rRN3*1gZc%j zjEQ8!4qbl8_=vkIt#oT_Q??MhV?dnN;jbYcf*1kQh86VLM*#6qbGn^q-VcKrN`NCC za<1Td1`ZHiX0cDN1pa)7vHHL6Fg6DwfN%=<8z@1lgU^yG!F*G}g3LoeJe<80gkl8G z+z9lU|7a?h)lLK#5_}1~{PqqQ51e@5j-qGqEiz)DcaXwyn`+#4>kD^d$-3-o7^9y? z`9{VmndVs?pyiW)uVq+VxuaTIK2@_hzepgCsqUT@wkVm4Dl>iRgeCU0Vz$8+teAQ6 ziIl(*&o&ursqFJ^w~C}46dcb9J2_n!aU$|tWa4!X7f(C{F#?!)XaR_a28$R91fA7> zZ=Ja_PyxiVen<+QeTi98DOj|HNH9?ZS%}ZV2};2nD8%nP#A*5|yfB3A0#=tb{v!Th zWl#%6i~|Fq+;RJ71}3Rb&o@ZfZ}7Sny9alV++^e;cdKVtkG^3|bDkNNDg0;>W3)(X zvpp9%z#YFqHfTmQ-zn}j6Cs=Q9no*=a@E;})sTO1AvJn^^r#a1m9aeO*E@IgNaPW@ z5)h|{_$!EqAVmNZ4@m&=aQYIN00eQ`J(vwW1DD7P_7Djcng6;&i8C^y3ir zGlk677Q^k!{lh{v)S1-%x*EE6P^JnxCQjMy3MOHhM6iM>Ci&}I+wUnV#E z0`oo}Og%K-t|hAdwiub6y-b$3+NN%Q9b;rn=(|uxKuRvY-6k$MQesQwIzXKE;V%Fl zep6Tu76vo%htu!=w~In3TRfwl$XsopM4H*(uEjzm(@FI`yvyy@2O0rAI)lW3E{3*l$VL`Z&S^G zLlwuyfzA6_DF-n(9R!`nqu})t%^KgCNnedU^sT@yhlmaOWrMBDs6Am8I z$W?pguqBFr$0fhNXVW=vv@iom0ixssqz9SF<4dXsi}nr)mX$$4f+j}--r!0aibsfk+7om zSf7duIDGbI;1|-?!;BYxCQl8MuZuU)9J)GDbTqxq=8bbVmjF$o?oeZSt%jyjUvQp^ z3)^1bOM)7M8Ew~WWGTZFleCasS}j_2*evXww2RMPu#TR}qatn%AYSKi@ytV|S8(yE zI>bCwpH~Y0$DKnAkO4&BXGjaITtk;s3l{Ah64U~?hQO16bNHiS0KPy@{|A6v<}YSL z$$03_$C7P(Tu&Z2|MfWvJ#7&-EmM@2w^m$u_{YcHUi2SUCbEnjsNOQ!9Z#?QS8?|P z0kWQVbpqNV=yY3~-Hs^1JcE1abEpsOHY<zNc%Szc;+K^dOsJ4gzCd54anP;X z#z$q!_4&4&T-ijv93W2X@Yg^Ol|pbhLkM~}ogrArP%wRge8i4Co&4j86}>G!KEk53 zd1R*6+W_5&^wI~Z3pd_90;t0m0CiY*11cY+2${2@2o#rkL5Q<0jgJO5aCXt~=t|!C zsjk8SpaBT8cL_iU+RU<+R0!r93KnD>0@mT|Z2%M@Fl+c1LI4j1k!#KW03`IDNTWl| z4|X^>IYC#3(1VZ>4G#AI`lP4l9`Bg4F5XbdPU{xjSZ5MH5!Ll{>SR<;INGVj{$B>xqUJB#=ecrxwJs99TIQnRMrYmcI!H8RU`*!_ocI>Xj}bH-g+ zR2%8tRM#$_pu|_1FCKT;1aOC1INV|UyfeYX zO6|9}hG9;SpTAjiBU^MP^mMwy7*|@#)b)@&5mTht7icX2DM1)s33$~OPCYdTB7ksu z@)jsTPS=MeRf0wPh6KYza1+6E0>0soMgllO0F>Yqo!RNhoQ`&T2|9XudVUy%&da-^ z-rxVQ=r>P?Kqj~Ns2q2eU<6IX-K%U@KW8asMe`J^D9Rgn_I2cUKH=l5qVMfVDy}uv zJf%6(65KM9eImQvEBfgm7416r%VC$o62F)R#CJz_L?e~{${l{-!<|UIm$ColrT>73 z1cU(e6_%@cobD(rIn`O%_uQ(~6D>;RRKvaUIEM%6PQ?gPtC9QLF@G%k3D7}ih2-65 zjmr+feTBaQcnAUnFz`?e01um{!$gEfg4rJjGiytk@^MGVI3e&MgHDd7g<#Xo`0|JB zLp^loWg@Ze+h1Jxv{vW3zonHz-xYW02H7BjT^c|Sp~J)TaBE==zO~R9v=&alt%ZLI zss5m~&>4sT;u!`~g5a9L-=+i$vJDCH4&ekr3&ELSHJ#H0fepe;xoW|kefrlVI00R%r0Pt`h3CVU?UP5p`yAOUgU0ep#0$7So z%SmPr1?y>L{LP!5myp-EtoF?ln=ur?TG01`UVo25_DOQpn+By1x6IT6%5SvbLJh<< zcn1vyWYAE6f}n?>kAP(?EnrNM*r+gTlytr$jGhDQZ z2*N~!kB}qQ5c+IbO_vr=KqBx;KyYPfiP-O{?)mqmvCL{hE}C&a=}r@ zqGfVoJ@yGA=u-P2omko~j#`Dt)V8!{&43rIrRRx#Wk{?h;xD5f2IHuQD~_dnMH#>W zgC=~9;a4^IUpEn>fD|CoCLulO2oG3NJy^7fNKg;vLKxs#h_i)a=48N)XQG2hQ8;{c z2$Rmhy`c|hGWb)1D;al`sQ6{d`tsJRyB8kfNzI`%iHDvMP9#=J}KwOyxa(6bqxRjbH*55R1U&`~(xYj(s+0N+;;xw%r{B-qag5EO z*jxTd%g$t??=mM|_b_eX6>C*eo5GpCOg~&z)fNa!H)nKJB6yo~>vQZg${}D}3lYt0?+LSM;2rSw*oI?aw zflqq^3xFSq0K6lV0-FI6GjyBUsO}cI8X{b$Y_ynI47@QXo+fjQAx6G+fNcnSX%@S-lI8_WZcBWjZMKM4 z`8iYt|A!`dm%&?{cP{isma6IGx?%D?Iw|8`*7@`+r3bLcR0%fr-OTwQ#Oz=q+~1v1 z%$}$}%*2qyZ=&-FJrc5$#WuW!$fHEWdvCaS;-OMV&ixmMF$@sikYQdSnC%vPQ$p}q z|M%V-4ge8An8l?4CFp4|KeUu9!bSUr1fxW72|>rdPy%=+m~m*Z*Ch}7Ch$buyg}A# zil-ZCYC5@{XLZTzgEEDa*LFW$!D%DwAXufLURP_P*^`$|E88GHrP*0fAOA&z#o$TH zjfdU50`!VkFSZ(}UmA$FW}6m~_~N!5-6`ug(ieMoGu}lVc^Z4-&|7JwQO5#^K_DhfO%(Vc#_Ha1IHW z^;Pgwz(bCT#D6M_*}thQ#-xYEN3b_^t-O3Qez2aO3!SgXeKtq@m}E8REM|f`i=#V` zKj zGoc^Q&5Gl0yn9b_6xm#D-Y78{I+}=XbckMt{VcyW!AMC+c`)~U;jpcbWqq`>y}>JI zB0n-BR=4n1;SNEB0LC3|1-QfM2mw$7=x0sa!Oy-93DXe*{8)f1Eq&|qNgVO`;uCXn z9PR9OlCys7ON!{@Oyia>j2{_!l#JcXIH&3zxQ>zPqUK{706lc(_P$}4rJrKEt>2%m zDCSJSk@~*#ai#10d*VX`^g0~ejna=zilBJ!C>}V4*=lExwx8h6dC}U*@ZN4X_FQ9+ z<^;*S83LdOhzS2QAPSG4=P#)!EZQ>!qChx9&__dPTv$t2Voe}$I5R^qv&bA*T^tyQ z>oC`=xqlftI)q!xES*gKuzlAtx;LNOHnu)1MwiCI}DNl~hLtBss*ToK~dv$HB`MZWaN<-%qrUV^a z7zuU-Cyo$%hJXld!(V_r48|c3Edlay-}ha`o1nGi*W|#r-v)Cb569`9KW!coeD=A> zGO^Gju=|i=l+t4ws#+5I_`Jul$CI$`bQ`#g$|czkT&h@iS@%QYwhBfW-JBEk{inH} z%9s!!9!`iEmp1&Eco+rVe?(d@Bmx7EFiR=|^X&^4WFO8E99ThDa_Q$qV0ti^2AfQ? z(0Ad&MAcV2pnd}QfjmTrYeJ_S>q@7rTiX*UprnyI=`mz!L2skuu!(^Sc~R?aTOryc zGuy3EV6tn}xT*1R!Bcj&uvPK`Izlb)ti0C8h1J<6m^PozC>H=VBvi3HdVXE4U8zr= zh|`{&B<@Th@p^}gCmwTE=K>ga|sVRWqO6F@Qctm`z8T6t&6Q=H$sU9IoRe#4QHId3^Y5kcS{hK)Yw< z#jLi%v{mzJ!K`h;MsRWTF{8|?Y2~M#=0O9wGa8Djm%U+Rt((3*WIxaqqwL=>UqO)- zo_P39zd6a0-8fs4k(!aasFsXLX!8HqI}doO_y3P;BI2ZktR_iF6elaR6=@h55z(+^ z3q^=RgzP9$WbZw*XTmX$z4spR|9sD(U*%9;`Q6`nJpSLuz3#n-+wH3F>wLcN*X#Xy z{vO(-9H;;cX`2D0Agb*J*IGIUE1d~`%R|ISJp9#S(Xvvo7%BP=hE9!wo1j1JsM&sW z&sCP>(Q+Xn7AEMLOU*3Vo_CnU`!Yf0Hje|fqydz=QpLKS1M!?%2F=}tDb2_X-}Xx9 zT1zgD{u}-E@jk=IbitfA^!9AtN&4<1vqcfeF{>6w;+w*U#B`=^&Zz8C*&yt3rNV;1 zAKPK**y|#$o_V-T7txO=f**5*|6`G&W^n(*1`&`5IG4R&QxRCTeTXqqglZy)Sbj3N ztO%fLiNM++&W@9!4a#FW0)m2~;DS)-s4HvVe(2h@Ye5f!MqlERIV(Y5O}?DeXifOBe-hsknofv0Hsc{Xs` zPHK0k7fzYmS`^TWGgPst9d)*gQg_Dki6M5Hhkt{3C>MmgPZyTQ58(}0o)0WN3V;;B zO`PdoC6_SnHxSgOo@keEa!$1IFW`UVN1;&%o|1}l(E$a}# z4wv53s2YNaf1m>B5CLumm=DO$+kXj=l66Zu>Q;D>&fgDROj0cb^^=_PzXie0obrM| zzF6}uH(m2No%ZRB@lS_tPZXa}7)UoMPIr3qxP^>Z$UDhEY-l#q*4_1{tfVYQ(fp~Q zciB>|da4cgc^^!+4n-HdFOjq#F|&0junyv_Vy#^lv@KC#-`29L`>h%ripsvt;Q z-Vs=;8{gZ9;4C1w!%VbaVeG+UbJQg$C}?3}4V?x|&w!5Y#7UvMA&K{{w>%c2ogZBLwyK# zsK25Z{NNidfgOgndi}ewLq{M4u-92xKn>zv4y~ygtlBrks0OGoK{3WjAt8BK#PTwM z8V`Utktj~uP(#jem#`}{s=OR57=n?2g&nL}7GfTvwhz-a0Pb*}W0rechuh$(VR^~0 z=?G@TwR+Q@k+4rrX39FUmPvO9d7A~D>{)$o7z{Z)H|fChjrTpB(%Klp9~}`qUSaKZ15$i?0T~N3sz4}?v7Vc#xqvqe#w1bCMZJq zR?H6o34nz;W&<5?GkePYZ;c6m%Q{>(KM=Ne>3csR0G%K!ejx+N;7|a1i5d#JyWjeO z%hJ+vGdf9RWjW}CE-OP7mK{1YZB(qmUdAKB&T4#ITxF9=C;}f*Ke)HK&E@MWb80L6 zvqpATg^4_sJgMO&@tm5H8anM#c_jp4NND8KVUazSXtlB)pCS=9z#Lq}yia)Molwl{Ym*4R0D;$@ANSx}Uq^jfqC$EN9Yj5FH_PO&m9xt$bNPwNj z;okusf)oJ?cqjmXhhdBL1HY>TXyBpDjwCbhQ-i_{OO(J(i93+$kav`vpQW?dZ^j4+ z{&9=|hybiKf(O2*bm{uTBL^wvIrM2oO(?YC;P~5>vG3 z7=Z)WM})4OAWwAXRs*x&(_utHBz4$`9EajSUX}G%GF(@x$h1o6$KGd$}~(&9YMBOizx7lAUwlwgheYF6%||7e{}TJ=Q@Dp zT7bRQ;p%~hAVh!y9$Et6Vf8;b6rinx{(l{KSPoPGhMa|@ASttSO{HMf4kE@x5#%72 z4*mlvSUeT{06bi93!$68^2Rj5bI3Gsfff_bLsHP(bAcWI2GuUAX!vn>pN7&B3IbASwZg?9dHGVpsAVl+kzahXcOKRpqEKrthF5gBxm%;cb2O60B3O9$u!L9?-H zS=Mue(1-byVtyMz^Ed1{bUhTuwN%>SQmq0{3rIt*$4hnyo6j)C)NpqVC*=gKQ@1iE zA2KK$e>uQra(FVnwaUGba;~=;5ox42kTv(joYCsSc>zBUM_TNT6=J7_xO(8>PwR%5 z1>naq;ZH%LQb7Zy0Q=I@!|6Wq=h$L<%qoet%d)rd1K{E3$8NiJ-p~yAE>6fRPsH>x zER5i|ZkG(Q*Tz{!Q(oK{L7;n2e;8UmER2zYyiMI>8HyR?LsX14!U*H~$U_E=i<_NtP)IIWRSAGu;_ zRrnAFn)5|(h(%J9@Pg)h@CN*hpA8)-R(qVhf95XH(cq5iu$LBDg^d5tJB-KBO%#EH z6R8+in%P+xrm)39;t1}!Bso2$xHRG&zb+vnB);WEA^i3Pn@X;Z61oGOprUa-yP|~q z`z@nfI0C44qQAvEL~1A@1C@~w?+}y|qjQ9q#b9Oku%y-P-*pck10{etbU<>@X!&YQ zDVN>k{)Q=4 zcb0dsBe#)lo|&pqVgeBtNf9UoDq^xh&d#5*KjQ38H^ zAoz)Q_&zD}JKNBHwYFi?;SIK);htI-Z=|4B4#&G^2&-sU@(mrq%@2DWod*;kZ8&sI z6=0mRi${Kc+9{|Y# z2W3=t;1NtfIyZDcc~T;y;n|Y|o=k~G<3xu=f{TMgdmB0N;wO-Ewz%pvCI&l>&C_do z)9Jf38@Q86%C_r+W}+8*2Y6kkt~41r`y89AfL9{lwqA4^vFY2%h_`?!D9zfs zRA%4*`-T_}1Ti=`+DiKA{e>Ej8r|fr;@C3R{{g#%knMsO#jbl^6tYoA)^F`xN5b_E z=jkSGv#de3tGhb&7RW}c#~mWkxI;n+cSyL9BEawxdb^Pu`YLWT&wzDH<`KdtP1TOK zvtG~PZPJ25n~3o!{{n5|2jl=2W}Xi;!M7o2O*LWF1|raeWx5EM z8E|m6kZjq2gM&jtfM-olvKbEthC`0Ctgra9XK-n;uDBPfE)G@!M}rxHaF{@#Kl*|W z1*|oX?%Xm|(W&l__^hCpHs>&-Kyyj|gaYZHRBSX7 zlH80EaF{YEyCT({EcAOeU0EUgSug2(xdYpMh*oeq9$`4E5)mns1i zB`DSYQzd{wOUQs@Ai6*}st28=;ITmMt%vZ5u6Iu0!&{I4pbbWOGq+pk>*JqNh8IUN zgfWFPMuuu#Y0)-kxs(z!RoP@@Py5h>XzEz=gazN#Im=lOZNt2HYg@*UgCU{u{_%-T z{Dmz{C4?-xRFYywm3A!~C`-n3nTH%nOoy>OW{kbo;p&Npav$hv!za0t;)MIaUMtY$pZO@7B-zVHvq2*CXhEA=S=BG3{& zv8E!hYU>bVUm2>5Aa32Cs=(s@@uG3KSV6LQEZ_v^g1Kn0a0I4?H^4QHDlg^Uaq00U zDcD!_8~^-A(SEjY{Ng5 z0@N0A`W7_UAe_721-*H+0Yr-Ss@A@{6*4~TU}U!ro3b$f-{C#SJuXHP=o=?>xr<_I$S;S5CjNN$iwgPhJU02{0k}& zl6U&wR)J<90IuTRt8BbwlDk7Fw`!B}whkfYTg>9p2c$!tAG|$} z_0>e!x`Ft&aEHhR+~FpGJCt7WRIvH`DS_L?ciU{LpNN25A&Q93(U8YDGksWQKke7; zw52n*pytxslE=6evrnH;i)h-v;VgVO@pLm+W0u2YsgqK7ROtuVK{1y5EOe}M};O1w`sD5kz?dc|YGFwg>+nN1N8 z1k0(EH5G(a+lUwiVPV1ep9;cB95ea;aKjTVyZ2dv)g#NzsE?L@et7spsCp9A2RqK@ zz=A#-eY^hTF z8%Bf(pOPitedoN3soOF%;kg=~PR`YO&C7bm6}Ii$O}}3~L|cf-P;2S0EyTy*#)mmn zLwA2iYvr2W{VNR#f6GF|7$R7R4*lufkBSha<7Ql@O4Cf`3rbZ3^MKYoCwdW(C00cB zkeoO0zf$IlZYb$2-7{rEL@P2|`jnpELooY8=8ed%h+|Y^oH3rG@aLSA8Z_Fhtw)>g zW$EM>y{e*6r#H@>;%>X29!t?EWKuXKIdngLz@1NM?)a=!cxqFcW}HH_@CbNV(;GJ>Vb!#$sFRGE-V=?BLPcigqoiNN(C0S3z8P`MpN4@*|oWd@2GOfzoFm& z0CXo{7-|H7C;eAl$pD+U^nTiU;-8n9~15NNsRAdls(loy4R+ z(flyaiX+Gdz|gCxOyPGqRY7fev4ItVc>%LHT@{bo;mR2m!fL9BW4DY7i!JIIjM&CI zs1Co3+Aodu*i-B@4gU`E5Q#<}K7x>k>(D6zIdl)|Q|Kl9pCU#-hKf+mVbRr20Zo)= zu#u(B`DgB-e(5tV=a5_eo7j#e-PshVced`v*bXBT1_7JX0}F#fT6D_ziHzngaTN%0 zvY0Z~^lzYMm5CtcPM9~>Ym%}5kaN+KZzbE%4+sG)Y!p(1_eqv~YdI!d=|u3)Z9^Q` zdh#%?rE0Jo7=7Ucff^I?1ODlPZ>S1i03P?sm%lylbo^3H)ZNeT&U>Kfhj7kwfPlIl z-`Vx#HK%B$&$sS#-iwT+(&K!}*{9WBJ;7;jP}6Zbqi5Z~6<(q#$DxN=EG0gyrrL9t z;y7E%Jqv_Fg5wgJYFv7Z)TlVQ!%sw}M8tjr8~)h}0jr~L64b|x>z2de$`9z`tG8CL68NaMQ4G?K^AE5_AMLFs;euK;9gg|ucof2 zMvgICh$SB{Iz@20dz`5P^2d=RXu@Ui7wNpKWGjlQo z&A%@W*y6J_<=WJ};g6s6Y-x2&YG&_gv}-i9X53EBK4VqHEcGx3xA$2!wYE`yG6gXi zx8$&Kc7yxZt+-M|dQ^RI-h&M1wXO|gd6Wn{ox{HYJp>7Y^#FPpb{Rkq7mEl$IRSWD zko$S-VBz_|93+Y4o!l;d+VOS=S?DJI^e3eRQS=O>SCX}Itm4k`4%0(PqQ24^wG06^ znrlR)A~%kRW?o?IuUJ=;6K`k(Pu;O6`?dHhL4pH^dmj@gQbB7+&!M%WBm_Z9#gUPQ zN!g62$+zCxJsKxhLvS(I&!;DAgerd-xfHzOi68^00IbxZ6i7jjQ_R061;1q+Vx%6T znh3J~TnbP}0zQIA{5WY%{R&eL)baMbTr4G%JCf8@?yn8CmR*7ZMpF|;mh>=rPLf9s z55*kE%s$@YPz;41<&I6-HBvbwpP^h5%fPp9u8=bvIqIl=pII#~sU_b#=xm5roJrFs zXAeC!p17~g#U6<_`?HJYkK`TAK9qT);CM-h+^LI`j@z)jh5$RA!__kn<=)d%OFoVY zd-42t&I12YfKv&ETedu5-jaOM$Pub8rlxJ)JC*Bm-+z@52J+?!891;ML1p%N<^n?U}EV21h$YcbWN=4W(Bmhqq1fU-X0Kgak0TBPv z^9une2lqY<*#HSZ;e64W3c#v8LyRFJRR6%!^?!U8Sh8pY-uTgoGg1LO-)&&iXyHTv zZ6NdaK+MA=KaxwvUgjiLo6p|1yCi&QHfHi_TWC@0u6gR>N_bVPu`L00WunR`f+`F( z1$arx9NSSRHK>v8bgU#Mscxt}ZDyZOaj#5us+LB}U~!pwOMv6E{VWFEC|lndN*v@x|h_ zlJso^JMat-Qv=%Ji*>m#8ktBMr|2xRti0K;20RfPdmL+sFys%U?C-WTPiXR&9eOcO ztGw%k-qx3#xssdd;I~e+kw>X`2^xKA7{50|9YVX(h@crr0cCG z81(?k5QG)~xgKCxJA$^3x96ibk@>yT<>EkpDT|kb zg_1rFhUCwj;Myw2RH=GgM%So!3_*Kt8%g>(8!QhMVxeWYdfXuh4jkHo>-Omhz#T49 zPM1-EA64ukt2W;O$ffL+zsH5GxC=7hiH94?a^AS8Ei+VIDR0@PE% zcPRA4gr)@w`jK<4Az%>^-`5q|IB2c#XlN=2Z3IFYNhCs1Iq%Y5l<21TdsVi`~39qiA_VY?Y(oI2E# z-sCjqQDpUorPkfc$MIF%8EnrLW3O|#dfXuh51?>|<^XrNP*(g~Lhx-+xQIJEMT1^3 zXcfVXJFI)Z@N@wA1^l7k`~r9aun#E{n=?Z6Bgu^4%eN;FC0lGa%05TEp^vpLLQfhP z21ZS?B_BFGBurO24Lk-K0CUJI2SPBA;e2u}4TDwNhZuzbvJcHNFbcs&^4Fp(`-gm> zm4qs-DTSpVC=tBtLl;a7-2LVLLav1Q>gWb@xg!96tPE-hZW@(k=|@i8X8bT$%c|Sl zOBd(ZYG{+pu6MMqWcXmtXKt1)y=3r%niB1*&U(khw(_!1RX1PtD3M7Sojif8br?`E zxi|@LDe)XTK*{UD7_&a6s@LMe-e$Fz*j{3eo&Mq9p&f!00g85rEKI|}p+U8sd_cd_ zT$uLrv~a0!(ACZG!1saZ;Ce&H$F9TqsE)FIrWp=OYAE6%_nl8(7sQABHW3^Pj%q$b z{aV2@DB#tI(6%dthF0*X%NZHW)d-Yta9lSVAP>Qs!4>qH!NUIW-C__X7(#^!g1mnj zCICVJd+ques6lDS$2C=hm5v3!Wgi0M;cw@LHiBiu*Pa-v3Kr}`lx^_67=Xbb(128M zHc*ReLcBVxzRH`JX`%ShXAY%a0|Ii#{`?=um-oritAK z*UTiBm!~ci5M%eS0DB$8zW_Y^Vci%kAVHSFVvG6m=HU;KB1iF+%|kXF=`M0Hk?;=J zy>BH(w`kw-wRj>kDDT<;5XkyrjKaM(97gG`yTqWaL+NTpy6*aIpAXEWy_xy8dOQV+ zQ*gFyBG~{8IUewLRA_&W6x!8?;rW649~RaIi9on}_L_>os?Eb?5=HCD2hT2j=l_)o zU>X=wQQ`Ssf~>)hh-Rg%q1uxym$*>Nj@C zr8POc*_NXqOy?;K_ZZ4->F~TsU=lhR**@T6%2iAo2h$kJ^>-+s4>={xr_HF;lA)AQ z>D>3yi!#y8(Q{JofGKus0Crl3e+7AnEreS9_?;Jmu{KK1Ry5Fo}QdMBa`_~lqEFWro+ue1;n9% ziLNwne>>&cq-=}0*9e{KgOHdw`LG71OvT7dmuy{EYL9vvBchpyV$8T3hp*xuOm%O%#3)JzRVyFz3qz&jkN{=wSv>09fgx zFF*!HhDO&^23Bn!VvG`@Ita@Cfeb7j2LSrCX^yraz+*8u1pO7<_h8YWvgeXIR6u~b zrg^M#NQ?rrNR+esEs_~#v?~dYGZIx&sd3h5CABD)bUljP*Fn!*EG;ViMLJQdUZdqx zk(rd9npK^F21j#)(+n%IEVr=!c>C@AaVxK9^(;p48NSLVj*hQ}Ph{el}Ro3OiMH{(ld8SPp~$hAe^9prZa6|5}aZLR+$6tJ03FQNR zh1p&O7D2dG3>+c{sPD32`@;binun`L9?Auw_=egL-;nbU&IA8zwWZDA=7$Y>RRRTw z&U?G23b4|M@VA`9W#a>bHUB^XQ2BxHsz~mA2K6JyRcoN$k;UueowL^Lcfrbm2i>Kb z$<$0Tyl93~yGLD$yj9cN9@W42>J~&`#bLK2M^BhNfVh~jL!@7dCP`mnxU)zh$B9^k zrzC|tRA1C}-gQstt|UACDn*Ks_t{kz*FALd-(?U_$4@(rgyk^d~SS_ zhW~8RZ~(ILd5@z9S1hR>@$s7~$=3~UnO?{82LtRh4*w4A5QGR&v_nBaI~-gL6|Oic z{I0GPO*>@X$nu(gTau&e634**$0g)?*5JjOLQ-^1Au^LW8PpUG$%%Vbld6wDW;v!} z%uy^97badj*5lmn_6ReCuTpSgC zn-RD)jK1SwN(NTJ-MddpttO;)!#<#f;lxUf7-nKq^zq#P4$iKOoT`{|AR|xfvh2w&Bm9vFHaEk?6_K@7yY^9!0nV0BWQhN9&Ly{WcNt z-t1m_s<%`j@O(}&&vs8$I7?i$rleMMM?!OZT8F#(S(D?laLta4uZ+Y4+=gLuw_1wY zT|dT5tLq`U816phx{U2|fSn%VUtt~wqM3&q0rPNi4><@GC0J-N1!{qAE?Ia&Sk6Pd z7W(gch<-o=U}2Sz5X4WK3#{e1a3vx5Jr6M&gAgoNJMb$ZSXx^O^$&n?!6kQSJiq`g zBwN$rE?!;pL^pRUB5Y3vI_NyPlogZD+@AF5jYWWt?As)@LFN_(Cl<$0rWA&h*eTyB zM3ZZio-3bAD$g98c|4vSsgYauss*2(O}N98Nisx2RYD!ni=e6YN;+sg+nbH$n+DkF zAFiHw_+y?JEFeKiqMvX7rN;oxu^or-P!RODrECUwRwN#->yG>UpXY-A)R#j+A_)X3 znQpdgvRr^AJp@57?j_w0XeM1Qe?T*d05?7??JWY_{qC(EYkK#u+B(EoPljqCaKrF+ z|Et@77lbhoAxPI)2p3DCdPff8;x1rLb*t0U<@;iz7OvKk4mh`ar#Fi`Ij_e$DdiYJ zUFP01#!15rN&}zxGn%~Kyllaxrw9-!)+)X!y<^ruyRBd>V!H8zR_rJ9&GsJL_eUZ! zJ!l4|lW2$Kv!*ScGk9pn>{JP@mIO$X#Ov685WrUV@NYp6<^=wSxX0$AxVBnUGlk!valt9B1D3IeK&p#C2S!p~U(sl|Sxb3*r_NFh`5 z?W3wk-5T&gyciq`y1Pn_6LfyqtEPi;{A7zk4#%3>OhmupN~*JV8UNWwaP^RfvDs z)dkDj2(ZvSTs`zq?)}-*k9RXuLeN9X1<&Ajjs*W}?qNB&`C-Vk8lV7mi3Mw_04q%j zf6F~wHa}46hhb9qt0Tb@#+i_iMJYHIK!0j!XlPW+x0zN$7<3H@kS3b-&V2Kfn)mef zj-o@iCNgCU=UmM#Zr&Z^tAOWgsAxFe<|#SzNc#Y@OZ=5K_q?JLdb{e>hXRW}GnW`8 z?Hbh0wPN+5H})DTYMyyaD+JbzR3!SU>m&6P=ilx>kbXK3%QpH~s?H*#30#pw{$3Kt)lxgte(*fiUEMAaJ zuw~E86O@x!)7l6<8*pxrV%O9Gi^h~x&p6rPG$T395EPxcFg~d@-zLcB{{0a3`0$o? zZz)_g+Ia2eem5hs{+p7i;hM@L>6K0a+gM~(QVm9K)4!ifygPohw}UfRTjH9dL)_a| zT~N_s00jvgf-wXMeswBXd@fidoqdF0 z=>7);=%B(vsG01=xGUr#qP7l3A4#lA(NkH>aU-aQ_v(pFup9g*fn!gZTj zEdEN1p}Ewh{uo5xU|`N?UY5;CG{b6zIruk~jC~?wTdQmuDd$wC5<40nfj$#uqEM=O z3DdA9hNn-buKj;;X*^~UvVwB;p8mqHlMQaZg7t1q=H6_XTiKK?H|Q1b>Y<$50QX)ID{r1>aTbvSUNE5>b@R$xf4Y+(QC-%;(BQDZRZFV;E%(~ zo>4l7G@mCByH|ZTTOgUsa{1a)g2d(`2!G54=|ISW4zMfJ>W@z14rC;HUoUavT)B(@ zNC3ytuB5FS!r7f>WShh zug@E?_U9NlAmttD8N34woSRf#h~c~)Q({Qv7kmdQd5-6+Osib4CaQja=&3a&*0-5s zr*Zff9)!@s5nzhxf+ z=pp7D0g`0tc>xc=5Yyt8mxCq80%+w3IvRkSp**Ak5c!mEFKTiKzw;Fu7I=&MJr$LU z%o20!JH&NNmr`0q%fD!hDAus9Z$vJ)!8$-e_M$#dOTDMPP)hJ*io9t-x_9P=v-EZ) zvx&p=pSp0Bjf;8VWFSYdPjRG3EDNk4Nu0k>7wWhXyPprR(?9$R*uy|H_RxahJlm1Q z1_D$&!4eTz8X|xyj5jS-7z-zBqbiKiF!ZGr#wV%uYOI!?3fejW3tgqxX7Ne)dX!u=L?UsQLNueXY=>a=^YK^;wqjO@{HCoM*58W^H!4KH493XvJ+FSFI zwstYK&kTX?RwE&obwQ*(4qu~5@#Xh2nqapKV5^7tx1fh0PQaSK*{x(ffF2%3KO%rV z1SmxG^O4|(>Oxe3X=gjb15u4V`s;X}QkJy3J$&c_Jt9~rCRowpj;6SW|tpY z{K~(Dj&k<~GI;sUrvt{@1)Bv$+~0o|2r^8UN>)kUU9nY9CzHQ^@Ji@p?tG4$WwFeL z14LRbkIvh@5{7SPlIkdxjgC#OF|q1dFA?wIbeI0yO)Y#X)p*)=ymz0*vDpbV$gKGLL-U*?%Z z;QdiQzP-rhdWIFX&xA48@JLqIlSsx6J!YEV)ar8x<4EW5uT4tDA6a4v`d@yT4=GOG zGO<4CgDbK3#}~Hsb1`hd(VHV-|o}92}lveu#zsq=}9VauDMT zNe|(nb`6FkWD!+oJNHf@OB~)-1$N*k575=!{&c;jbjWd&N3edb<@HY*@0dsirVdzU z**36$zVLjfPNsPM84intmiPUf@40xpPmeq$ZhRnnO#Q)CbqPg6<-@L2PA&uUrLs}6 zZ>Cu@$5y=iGr)}xD{X=9e)EvwHNE>+8WsMQfrv3euZ=ZKYnAMgx@Af7=e z7zpr#xPXAQI7{*<)L6(2@eIl-!RzUA4s)qZWcZQeGi@(_8>SU zn8$;%Hm#rL6#pc*HI@emu-83YJ@W9U+|cg`0Av0rsnay#tgdImJ2}!kT{P|b+`UL?uCW}U7OoIht5l+x zn)xVO-gTdcGLJV`Qm1zBuvQwQeU)Itq~|lvt(=sRTjF17P_&gh+;hkFl0xjX4gU`D zP%g-7&rO)QIfOS{OcY~Qf~Dg@XaV_u#WrjPDgYathom6;iQk$^!AfU=e{LH>34+0q ze;@^@`C$6y^8tw9*-VoSpdrKRL4Gw?uaGcO1B0WoN zcML?24}s`0Idfz1;T~dfyCT<*$`ek0dyJW|eZK&Emkw9YJ4B)jPo*K=q4bJFft9mF z-zdnl6w6VNzuPxlb^$r~A1@#e0}+6k6(fNXyc>#NQzckw&*-;|LyTD>R1-lQh5*5z z9ts>^EOZf}{$dS2UKWK|$X}sOA}}hHfH26{8m;bJ(fcz0E=Sd8Y#k+?hhWU%n6tgr zy>~1xk;q>i%ZXA z59-(hJx0gvE}XdKjrBF=*l8U84d5XN5J14gK?r!b*hR3+d~m50KwSjBo5&uMm149E zzeNak-i!Ne!}$Hv!7s~;mAYSbT6Z~urK7L3NdyCi*LakgXV)RcDz&^p2qO|@B#PvC#?}V&dmuW&)ap=v~urd85WLPcrhfh?K_d-K{{INt4Oh=RM5398M;xBx?1It*AYuqH5H%JVh~GAkf1DD6A;HV$ZiZ{HgoD5)IG-+L zaQnN}LxYp|t?)(MH3Q18C*4UG*vj}ZPISQTTY2#!2AwN$lvzVJ`%_3<7$YwVgFaZ# z$)%)x=zv`a0+EkFZG zEis%E7BpZnII!49g4#0zeZ}P-P%()=>WX%85yJzNBh>OSv}tsi(ocoK0zb)l3P8Bs{^fY`2EG-6m@O@ytE;a_7O2BMgUa(e;u@LPZY#Xv^>Of^|* z8DeA}PHuPXqA{T=ow=8>mn|ehWfx7vEf+fv6*EHV)^k$(Q${WkJ|*V4`%akPWA`TY zv52w9mpSUbY{k{d6iRO1w+iH;A5a2VSX?uZgS@Cmr`K|3xN6G~qZ~j90yD)wmjld> zp^yb?wcy@j`v7j6DQfBH#*J|(VXQU`yOcyW!vNmz*}(gKN7DSTFS5m6o*w*jf+zS!|ODn zu3vG|4`{{mts^XS3|G%Q1i1ke^YAfX9xjs5k(gEB#|Hz*H?+1@_60E_0LJ`1vB^u^SazZ|c-0 z?x~;_(aAYqKP1ZHE@F%UdH7Y(sI1}F$ioP5|HINOTYw0BcN#hO=QH?LxX0k3=DwH2tUk)dLX+?gk^Co{T&cxfq-C^KGsmbo#_PDX zdb0Wm^g`;ABj=IGY*ZL&V9vvDJ6v8!7~4^2*y|py9(xGl11RjFCBPn5FU$*nM++AB zjReqIfT^EM{>`5k7XI^jVFr)^SZM{M1-8$^*HjBuZ6adS0;tA3t?wUb!QunLVut7= zl}jaxUT|Zt1#H(IsJk>elZa@p4J8q~p=j~>K*qBDhcs0*ZfV`q6l)svmwr z!??9ItII9Y*70$xIJVcAW2c4qcc6zLM}UGJHWj8VW{WWk!BPhiEF5)al%oy>kM)L& zQNTkq+yn(YtXo_>WIWazb<4MbIXvljiqoS+;?01WBZ3Ur`%B6m%7V5Kv`Z#js7cewNcU|DhNmwc4|$2wX#?leJsY`^Y95>gI=c-sh5YsT^oWbAbiSI;~A$s@q50ZT@OKjeup zEFJzg^F+R-y}3?o^UgwX=-c5&7?T1yFk4)ru*K zC+aYg?u_JBRLE+7)rT*xB<-VpKd+vY_;|01K|)9Nqbm9auPObug6H*IM#SSjR7sr1 z+M}Y54wFIHJ{rJI{aMOy91WQi>AWL-l4*HScr<1^cA9FSg6aZ$H z+6H7Gq-646lY!r|4KdOVQ56K)|5OH83D(IkV9p>AI+}AVAd`?Thhm`f7t1*`jJh~w z4p7htRp za%3U~b@<*l{7wY8lDjUs0`$2hdYj+~IqLBMuO8|x&H^GJbZrkz=)sG=qg)aT~WgQ{gzQK9062Kz~5pWf&>A?IwXf!hvaCI0Z11?FX=-sE%cLo z%MmOb3(%FN2~MM@V4BDMkpc_rhM}!q-zrOG-_X#3bwdqj3gx?kIf6T-6|*lL_XG<> zHG7A=)Nh?L?mW--vEq&>N75J72ZDiI{oQt62`&DzgD>Xo*|vI#Z+*#`yH|lz!L*R| z?$imZald5MFR@(Gd@H4i9Dx|XUe`j3F!(NTO%-9)&LL0)%s~Pq83&RtsE&5DIx7Lg2%9kUW)H-@x}LYY$aX5oWX9gQzW_K*onKZ`R~;`t8^-NVr{ z&dm|aPX$=$8~z2{VIT^3$R-MKhp!fn1dGsuov2{Z;y*354LjQ<@9?7qAYiySh4%tk z9nwZ&??k3u~9+ptd7Ce4(vzpf1rTd2CYIr(1SL-#e=oweowr{7E z)q(a6Mb+c-)Apae=t^boqW+r;hkoGZhlRcG018m%n6{=0u+oU|w`{{@vjb@w%iieA zegfqh{?JnR!Uc7i++AofM^6jzKvf}3AUCx{Tc2=&))b@5AqgUstYG3P=yh zly!10FtOcWpLzW31)(#|b}qEoT|a=muHou=haf@#@eU�Pj%Z52WCmYv^kWMT_G` zF32Lf-aL3=Jg7`wUf5UT8(nYlM2S0)>X3JooS&t$nAwT`DB`6xg#~4RXHZ=Y@eEKk zg=|ngnQ$-ZJ{`iS9F2m8p@o{lU#=cT02P3xO+Zo*F;}&wQm|^*5MwdvZx~bBYBjRGa6ogrSNH| zI?O4p71gX?lxO*wNtwBWi+Nm{X40s&(Jg3Q6|*#Pyk?tH)Cz^Ot8u zjvocA!#0&v1Vi=A!S~iT4^3ftvKTvU!@q+(1OWmR^6(5m9)>L%hl>Y+m5v3F7JQF9 z+6ey~EWb4?%_ig*$5(aO&Vz(TwXI!PLGhPzN32Gr>10@TMH0=bGu$I695lU<=_hn@3{O&!*E zUTis=#VA~PI5t=}m{`T$(~FZ&TeE36|I2fkyaum_+3DtjMP4rxU5rHfmAUnYj&ziH z(HqxjjTlx+GpwE3sKEGnHv# zg8PWRKJrE0^oxL?@t_VtuTyff>p_MFwT+b2xp`N@+9E)u7& z>83T-exjv4Q#Ir*eZ6=dmX8Xs(?nc7@Nk)}f*+p>R*Vvrav3Opw{_TKR!Ov7mc51F zKS=-cW4B#9Z)gU57boOhh!Y;y?UF(E+BnN-%8MH#2z2l14?|mrg|e(*NAZ=~AEXI~ z!JQ8?1AJie+?bECYkK>y+CIb>B|^0jlr8TPE_?f<<^$YcxO`74dJhBbF)hc~qry>IKyjBjZ! z*jMs7VUEu=>5j&GeMIZ^+@784dmCa5$VO5!tt{+icms@k!i(d&Bl8eM)ug>vhjW7I zZd9D&{EY2!LTueT{9Cj`q?Qu$gEA7L9U^~gM6e9)5Hyv3vkv>4X7H0k>7;Lel63KH z+QZ0PuIaRuo+8sN=A3bd2RR9?VfjU`oj1M^G<4q{MQn z#ow~-49U7nojCZ62>3w{pnp&r@PqE9*g|l0w}IB~Q@541Zr>)sy{>j& zO~42XkzeNPD+GKbywg z)yws!H~mJ1b5wY^I6oMN;9tkZgGPd8LVx=DImp^ksTs5h_YPOxZrxk6e(y}oquldc zCtIUrLdiE#DEAp;>Fud7c6jaad76=t=oiybLNsF z>Pr-F-I9}&msg@87t1^;Nf;R`kDMja6v#ZenJ*WA>jsbYuiPj%({=5+(y&3`DGd7e z*W;`sz`;3n>FRayqc}MAxg6_oaPZQ?j`lRN^5#%VciIh;Cw~_QhY~07j3ADK zuprLlbwM1Yj37?#Z9$wcRY9CrI)XT64+U{LE!GL*ls*F=;$DIe;Fn|v%5I0sJP8SL z!2jz!F8E!v@GFV>91?>45pv@!{|~)TMQ1ux(RsRz&go+LIeuv#8|DZ5;>d>PO$)oC zAH1N_+HOfWVB*dGNvWJVQY@<8k4&t-bS#ZdzR`;{+ml|Y7(QB>ae*VbD@N~X0GUkJ zX?nvlSJ^TCs*Gf|oD6>T(H--XT=(N!dBgL^iO2IB?A+}*!c8j9;$+u9G`=S8hC4`i z`NGv2UjL6gAHP)IQ;oh6edD88(3hHSu^!!zx_vccw`)I2^%cbQ2@GSCRcowmFum*g>OrhOLecHuVrFoZ%y^KoYeG2(Zc=>G_ zWvy;{8w`7BYqcx2jdt<%`g%$Y@ojD&E!q5alR%D7GDFAC*(bPag?Wa$a+UT^=Q5&0 zeJEay9i5rx9UtR?bMWYbp7Hwm!XEXciRqpV>XV4tRMQzRzSK)3XG2QjpMQP2VVX8! z-LpK&th%Vd5UPwhw_5FK-iczaQ2*KXlZ~Dp)$09K^C#l$q?JdqXM<{jOk?IZ%_)D@ zX7afZmq6SgyS z&w9JdGi?(HogKl{K0;L!Vf9}u1sIw@0uzVWkejLJ#3hpVDh|=eBB+ly@*MhYe!K? zgxy2I^G}sO9xT{&A}W&h)ol*gE}DUo{_H5*i~S^};Yu8x8i=j1yJts7>)?(DN|#4|i%Rf^biaiT$po2}x2 z|Grq0$*KsJbkoY6eZ=i%niOT3?}V?OO0-FOsm|0}w)tknX_lzg(KaePwWOz|nNvbM zXKPQ(-oDBB`N@f>A-W^@_xTyOU`)@yPaOEcO6X1n9X;SM^sAkD?^t+y93hiTWr-Wvyf7FszL6r19eG_81C_279ZDq zBTSUCDT*mD#An;+tR$Dw)q>3FYhtbjip~!Qr(){XPe)1Ne=L0u{a6eC#*%4M z@@HNkU$Ech=@UC5y!%FKT$OgzrNO(A32$|pt77ilzu^-{`k2wI)Csqiv@Ar6rHzTq zi2sz5QoN{@7|yGGYWvs|ZjX*7zcOOz7Jf&dqo1V;LpZNd%XNfW zMHzK0zsmSYft+HQ_kp^}xkA*zEb&)zl!+~}D6aW1+#3cz-2xt7NvoMSQTh_XjjBa` z7HT72m#-D{kLcfURq{76A{H)=5$QP=Yu=w~UV0#BWO{7)0*g+JN%i!h*h}~4spYa@ zJiZ1W_STFELoXNXRK3(wO%3xjrT6W)PsZqX+tu*!$Fa*Ulsb3jyyRq0>5M!!JRRM8 z!#?@hc4e9OnPdYNxt8{@8$4}hel=sVEC(j`et~}GTWKFvTO{Ind72_dl)aQ9^_V6u zdsXE;Kf&9?e2(hmnE&S%wcJ8A`;yAB@q0acThnA-Undcm$Z9tT5ss__L#bDlq3=>; zqsH0gWzL^0N%kmpro0cnm##+d-;A|Xy4tErbTn)|v%&BAhkBbkRRzfvWz!p^qK*Yq z_FKy3@mLVKYo6K0lvvmDM&}av$p^O59I3rbP0JHusuW{n+@4#c`-UjCe#47ydy<6d z48qgVJ!7)@H+j7huAB|x+Y&x&gzu%P5fA>COjr(#lWcle7uYg2A>fDqMKFflFhn;7 zNtt#lv`kiYYtK6)evdI$0pbd(tJe}C5X)Fv1wHNEu(LBt4+%#U5BMn>vgkyqJ@B`4X(SFOy5?FX zJ1Fg?l1Q3(|y_{sy~jFWOTcQo%H2bPHvkTb;ON!yRckcPp97!?Q&UJ`m>1H z@ws&Z_9EUvxw^?s+b6Ck%sEut*hn#t(>e1Z$)Nh)>4=^&?;;WP#5wWGbi?@#a))R4 z`=);`vQa{`(a;8em1Hh`^u)&k`C4S#2|Ab2S$-}89$pEKcz&zR54RK+SqoQg74KEg z4;eSCYeV>Ddhl&}!9YxsaQl3kc2r~p4Q-HA-Rn13X)LNmPqDbCT-?S~TiJ4TtdLjt zPS53sqx|O%e`WV=Idhh@EW(4wyI?EgeY|Jf=Q&Y&SCd^)^2FN=V_G!ST^yO}Ike4G zg9=YvBR0YtRkc<-apcZPr7cE>N1lc1-ea3_544c*YHsTL)&?zCrC zdtRN=(JWdIzU#8~#9uy!RQR@L%DMaOiF%`w+;LCYpNIFF>y04AnC`GC3hQUnhkdzn zq7Qz!EoEdJsHW|2XE%K~OV`(M>h*}tJwy!+?FYfi&7Kc$3``&1-_9-2`SSSJ_QK$C z{Tm%R@6+4{YJ!}!Y1Z{|e_&B(xek8A^!dMy){TAWCsC0yVsDnxuWivUqhL4}>~(CS zI`bLUZjm~i_A$-u|goxPC!b_a~FxUGyya`^5P%F6- ztdT|3H^cpkfaX92he|b*+uebori2KoH-#?LvV#Xth3qzJd{1&ZM&+TseHC?obs3|* zfXZhwJef$nr+cMz({pfp-PJB}O6eMZCfm*0WouC!TpOJ0Qa?CU8q$B|i2?!NOojL8 z=~BDhTHSh!$~<$3yf4s>s`f*mOZ~)PA#(kq zsA56PlvapIRP|P_n&FLEhf;*^w7k7g;u8}Y?c;yH>EO9$(wkOU2NcDL>feIfa6)Kn zAMVwgNBN5qTEZ>I{X@-ThA1s|dfgk7?u#n1avHx{sXh|JvaRuxx*6}#d4bvq>1#mu zzfQjOs{N>>j;}IAMd@ZdcA4B^n2naJuH}Iq(i{3>Fm(Dx38tj z3VEZ+Ot%-NH|~9KOYZqzshBT0b6X;4S>olba7_vJiYneDk1rSr=nJmQxl((uTkIJ4 zmFtY%t;Bu0Gh`$H{DKtl?KC}e+NZ*lKx`!PYq~_8{uYv)7s(3*)23kr1Dd%g>F>6;24FnJ*keqZlNY_)nF>uWeVbqFY4vHj_rVOi7r zh>T)V`>f=y&2RU8T4~)uT#G0U?eHnAj@R?IDH@5Fw@c-3tmH|!uZyOh+6eLv?gi!*5gfxt1bUBcQp;Du} z5s(g%?gkM-LO{Ah7>q_b)zL8=2#$Qte*T{~ytp~r?sI*w`*UAsFLw5<(4@LT*Rplx ztb}}?!Z7z_4^<8d?SKCc7T{40`0>#D4_w;ctK@OxA6nVfUab}@Zwp`lGgA( zQN8#1Y1<7YtTu@hEu*GD-HFAm2dYgT#6v;&-Q*tV44Fmu#=6P;lE@#!3_E{sWgO?< zG81tvDKcsJGcd5f z7u=}+me=Ybry6>OmYM2|R^A&u+S-~v8&oEET5QWzHFHdUNeT}QRD+M4l>zM9AO*Fo zYWFsLJT1r%>3BH*I5id^)vk=Rr-3JTafvE=M265}NIlGMYDdZ&ym(!6Jj2)QLHx0e zpu2d>>4{juRvOQB&0UR(otYx_d<1|mll>O>YkJGoWLfcZC_o8wXPM2JY0ev}q%x2x?cp@)uiF}!Gua%U4GZ}{1LYF$nom3-=MJ2!L4c&mFA~(d6S^&vmidi|0)4XgJt!{4s{np1eZd{YgB32cAl=|Gx4;PaHe~`!WFHBAJ^xX2j5s(FSNbQ2}w@QJ({GY>k8*5xmwlND3UhCq1i14-C7S;$4Xr3Erp4hBK zLHHWay4yZX@I}7+OWR_|2-N6p>4sagdn6%Lthwk)Tdxc$Josnv} ziepcC4v_d>^WuCv;|2TFTPgCNxk!*b&zECQl^n(6lFHBnqy|p{NEK7dvtTSJel7v8 zzgFGK)B6Ub+0+8FUaf+Y=_`({aA(REw-sEhlK(Q$`gm{psz{n2gF$%~N%u@| zv^*y^UI~8r4Nbxz^x@l)&3+qouGqR7L-}8ZjP&OW+Bv!}ki0jsZD!N%y;Zu7l^LdI z43M3x*zxJs<8#5f+bF!{q*S_4&hjIcGze^2{3!HQdJ^&{yDD*cYJ$%D$HDJZG;DF&O&E5vB`GM6C1g7y;rAdQPB2Vj0RGFE_zS)QMsqT;g z2D2-XzKBi*IJP1bI1p~>zbThd1vIQL2O>LfM(-mSw>y7gS5<9Bx;`sk&(~FgllIZU zBIjoE@#b89OoH^Xfy=ffpUT-?&aFF1&mMX>OG}yq&c;L0=6N-FoR77U_U~M?3pIIG z0?4C7R7+M;>Fp65bOAq1x|VTD!cb=y4~+J-Y`5Lq3Ip8{V}}1k?o*l1TWQO9UMuJy zTv|Q;tD%Ut(_6~|ufB8m`x0m9O#VHb8cvHV;B4D$r>G%Cz=V9#SQh`>=J(tP?Ag(v zS)pBt>^r+=_Aq@Nj*|uvdC*z<%B0u-@;X@LM|@b`a`QBhC@wDiy`rnR8=ZSVax%dO zA3Ys``zK@qwH`qOvxs4d2Az|#rM;2eN;!+olX0+NMi8R|pis@bD@f@ZJlEA> zFqp(i_wXJ*7G;K>+(%w>Z)t9=fa{ik1>?w+-$tsa2(&GFdGB3j5bO6xbxIRFCS<5& z?__oFmm8~0KDU>rc=um;An)IFZ&_JH4Kqmm5#=|ysiq;y|MZis9@@v_4wrhRIzbSA3(+9 zqL0R<{=fB38TQhh_tB--#7c>2ikwnE{5b`u0pd4)4CQOS-I;tTa7@8O7gDJKYj7_r zGHrN-C`H$LM88gqq`YGdryPo`CVetwl^!bDW8d<9PiG-yh8C&k`(T1{B7jqdZea~8 z=a*#Vjm|8u?QXO`@Nvd{yC z$O6e)+f<-KE=1(-Fm>M>6O>CsDAKfp=PAfELaZ3<3D;B_}_2E1B;r2IGC-cL1iLsEt!BlXtB}o;-`-75S9GoqBc2j%F zL0(Qs_yc)?3=8D~|JP2qb;gf+Yqn4lF?(rEg}g~o{E)?!sM94NchkkDG1`=e?4;=v zWmB6rpt~mM@_9c=WbADB7D=k8BPq3=qz* zOL*4B5({WYAhK!9#@dtf@7n7uer2Hyo6LaGA>SA` zELfT7i#EYYo66mA@hs+A>qkAx!PLdx3FgC?AFG^ z7`OQ%=6oc*+0bPaQLz*L5KTQ&Djocp4x?@yt2iwR&0?hs`Sy*|*>L<5XSx{(Hd|l) zx;e@4Fi<88&QBrW3m*t|5K`vx)5K7`7rgi)&Vo-~AGb&3pF~<2RBO zg~gy5zw#;0T-ib9^<(bMbw8OwBPOP}UM;Y9Q}VL|<)FMKQmA6A1~gpz?nYF^+F=>z z>a&)^*i56x7=CAj)@5fwq{8jWV)3HxrBM`;cNm(2TzS!hc41)RT#lN2G(wrF7jmdd z+zTA=74vn$My&1HjF7TmpO)a?Z}2+6W%Ilfn2wQ5NkH1LJ1OWYRW5)u(E&ovo=>z{ z=)OLidJYq``I|1FtAHx@E&FtjL(Lr$+L1n}9)l04%fbR0S^U>A?8bVQb0)xsXoT`n z!lyCZg$jrG$&FuyIRPKVNRAw}9AkB2qKkw8GwhZYOSb*`Qe=99Pjl5f+Db^u3e5t% z`^$zYz;C}`KtLCQI&JO9B68An7gBJL38jENuOLY`?bJVB zilu4#r4rHiPJGL0SFCU=&5X^#3Ur1wvJ!Ox zWyw(xQgyI3DO2$-DhxNDW!j6n$Ygbo-)9QwLDl8SxZ1S#j@B&RkqovbnGyVpEiCR> zKz{E1P8r1ihuByuAZxw_DkqqctU%%{Ej;DkG{wj=x~5$+zihm65;j4W$Vvul_YP+P zMjkGX7|)c9fk)}_K(MHmO2C0QY?Z0|KAK2Cn1o@}V;AqP*TyPH$IwO@yW->M;An-B zddqB%)SyW;zq7cebD$u!HN_3>aQl82*l#pYm$K;grFb~MmO%4FjS7^vRRTpzrhmLG z&`HoTSjh^Ea+BM$A!3HPt!BxQo$&+N1EE*%E%}vs<-FSp^(J^$6^6CMd7j^BjKIMx zx!a(+=@Uu%GCd|U;OnQm9JhffsHnKTE;!ULotW8PxISrNgAoJxFp5&>PX%mx8WHf6 z?|$k}$Sc66GrWpeJjb}ru_bKb^#WtM8>mPQ)JnF#`+omL3i3V`_9Y|bG!}O$|4$8A z%Vo-@o6VvE<3{Ct8DCxsGEosA0rP_jCEu$oDs%Mu<*OZ~btSoNrIZag(UCwXs4Sz3 z#JgSAV{((G1lO*sc+W1o9>7@9sF01oEMPMG*sElHvEw{|v<$#-133B>7177c4)WcJ z0^0Hw2aD)_j{#m?Bxj-%iuvUqu!;brjo(l<9eNW*^ai1p;=IB?0nm*7wbm@&Y$i_I z27X@`SDd!$C|lT4$ymA@dl7DeyKUKr1_(p|KA;tUh$2>={DaoZyqAY4L=#}KD1^*U ztQuXD@Ce|8d!H#g)-QY&oAv|}&HMCO?7rc$1|>a}61M2lB5!zD1j-L9Q3V8lO`IHW zQnBr2=}BlZ6npk`2fv4_#iL*@zt<&}f~tYDMbR^|~2?2|%-&5<26Upr+3!0!lM zib0%I8L$&LF+$a2YL~Rzg&A;7;H}5aV==FJuCv?Qo@?ZefGAU(0KQoBTac}|et#Z@ zJEGiL=MQg9rf}@-y#E87T`n2)69Cl3?dazJfvYA~qa(l*0_dTInjK>QCvk))XTFXr z0w-LN6~6~Nj=ryBFi4tuXRD9VKy#*0ZY^AMmrTj%k?aD_^kq)Wd`?TsIn^Ddb-$GC z!P~%QQA#ZJbxK=9j^-7WJfM{oB5)Ti2w2GT1del8_piehJYM~Hbkn(#x#+<#QU8hK z+%EQ}w|>`;Wb8S>t$2W2Ps$WdI)=Oy8}@i>jzTPH4>*&UXZn}yKgI=e31;_=P8X7T zKN?KyJ-*PGAT%_bpC(2Y%%lv<*o=2 zitpDE_ zZXHXZl6^V~XgPuhCdi^OL|$UpiA!J)vcKNkOk3_sh7;}MhsN8rps=f&Hky#YQ-?y_ z#pmN>XJ?$?EJ?`&aHbI>%==HmG^5P-r)sW3q0a(I!I=S9l8Q~c%dfOood_ZqOhma< zBGv*oVEi@u6FlaSzK0jhLN2>CC^(V(D^_rtq;n$>9Ag!bR8ga-R}cluaec3+*ukm) zp^M&hs5mNO&YInM$?VX>`hH} z|66Zqi@G_3xS2De0T(R6k1sO8CJdOT&SM7x9UiM}#`f!gljpp=Aimu|irI7Ix{z21 z3;~e>Q!_%j)W4=awjS7+E*ja8K3uBmE)*(mK@9%T=9z>eEg};08p?z7TZBr|AfE#a zmyaIZJ}}fu21*(8!=q2CtAGLHCC3^9NcwNds=Qkm)Z)r-*9KCH;@(9JL zb*PKN94vL^@+}KUT5B!KdtWe)uN0dAJ^x_)&*_ z6EYLW3DBy$Tntu+`8+CEdP+Ua?^f~Of$U~6qyg&=+t`FQpRkWQePBJ+6Y2DF~ zX#SDufkQC>3gdrk!#6nB09GlJof!w@m?GZX_!R4Z{c5pm4|D0IAYrvVHb=??mGrpe z6=Fp^@46nS3+!+pB4Jzq<;0AAGRBsr55PX-z$~%@-)sNtwmuS&NLkD03X!`{gRQS; z;9Z9TYwnOA{3T#_d{aPI5tX0*iRzznDqs1rGkT8!?CMbwGH2j!gRUcv#p8F|V_*p4 zSn@~cCkL_d)Nz5FqGvU@aEOFDz2h?Vqfv_soJDN}+nc+K2VT0K{8`5L-8QFuAG8;I zRWj!0T{OL@S;y35Jhff@YLS(pjxw8<4M0Qs_H>#l5qJ$Ohu?;~_x(0X_ zu8V_jG;U$_@k}1*rz!D<=kO<%L=!L}b2LiB5f8uofGe@Pzjbh>*QTPR=)Zyt1rvzf zdZAmpNMf10NpnPSB$xeRC4uHVtt5r{xM$1PLiWCIgpr_ZTfln7VJH_1TmY~mZh@03_7Ua@b_^Nt5zEy_DMA}xa+Dd7kzlc))e6O}o z6_%9cfwhjv>f`FHVo$AZFJWJ{?I^Vt?#o%Uo5l#Ia&rk~jD1$NQ;d<1xXECrs2q48 zJWl-ZHg_kTesX3$ok3lua<+G!>_yJ7-}evbs~a2dCT6{#-A`JE$I0ve?n*4VWV^Eq zUU*~;N3cQG?G5=7`tI4QXip)9EqkVRg)~x|YFmvI{9Idg75t!oOp9N5##QUe;6uj! zmkAOaU-his{OwXr&PVyGKQD-Y1$AVtt>7m?MWe?4N<{p{Z)%78Rd`fxA+-TC$MF)a`#_(maXVjRop0OP8H^wzs<+)s(ouR+MLqeZN^L@HuwLdap)* zP+81zqxGvLn)K3ReI1%LHp52wDvx60lV%PGAw9-nqu~EDtMY$seW!v8m_s8C65H~QFnNGCRxU+?j9P5~@ zvF%Q6*9Bh4c10oaQEd3Iivl9~Sj6)~gleS3j4oTIYf_Fc3&{I6b%ndDNP=RZ(!G25 zR0rMXk7e7IKDXSCDmX}sq=t4+=|{gP?#$j0Ji|xaHn-UmPSA7`yp|d{yp;4~HZn}h z&~eG_`Uu&n^Nxab)AdI_fjqa;uzN7?uS8Bs(Ky(I2l?bK_axy?p7!=7zuQczXjtOe z_xiW#F0?|D#S^XO5s5=G{x>0pHZF9gX)4+IlGP$1RC+bPHGK7duH00&*)TNU-6osITZxOlG5>Aq#!Y(rGnfzAy z1Ss1tF^q?X!$J1LZTMF0=7C# zOl-jWJ3{cyk1&G(GMgisi=+ioo6UenT4wK5ncd}ulQj3jAk|g&>@&LGF|h$O*)D7r z-`K6tJ=uAsOMPeW`%G+JbMREP&62@FxZ))=E6Tg8KLpb3l2;{*+=lpYBXS`0$8KYI zR9Xlf5ao|I9f7%3Q(v85dywxztL$7_i?qvS-=X^qT)QJk$(@|D z3QL_|nXs26jdJPrBem4ms~Fp0FpB0y1sL7}2LD1K^9f;llCf38)SAD%DCH~KW&gu;X*W-bqwuHTvFVvm`X-0!jTuTdnS-hL0L+)C&lBgrw*`HR~4WH3WVUdq_*?cvH^ncUPeXdPUieRRP>P}_E<%)Dy zH$0@@UPC0sE#)QKd(e`Cg%r$FazB9+R0h3TayfYqT5{Ll?S5FQrQz9Ju1hZCuBuP& zFp#RHX`l6;GtzclKa*zaNq;f>CHH%&|B{y0uir*sT~RKpQ45>Zyt(d@;M`AcZ)l_P zlb7Md{`T_i930>|%h%G;vyb-9S^ko0bc8M=GqeM1Lfe%IwWHmLKU4VW&EQc5-Yc^s zVhPav>LoG95DCj2W7x}4o|FlJHI4oa3(Zwg`M3&>YT0ZjpCr9jCf+_@$qRjP#l#+R zEc8Q-pqYD0kaf;6945LE#%5M*Kt1BW^OI!db@ z8D+g=LT7TjQ&mKCSJQ)hct|=!sw|{c$WLsK!wfl~-u-$UgUw7x)fc{>yHty2mt3ja zWhy+9>Z*!YRJ|5vf3lsPcjfhs2GHIR=9F#gz`354)nds zH_w-Gp1$1fF`(rQbnl*q%~^@tgs%6NDCU}?$X(6sVl=P!n|0T0X6*yt{QfCJ5IOny zm$;xYz# literal 0 HcmV?d00001 diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc b/flink-connectors/flink-orc/src/test/resources/test-data-nested.orc similarity index 100% rename from flink-connectors/flink-orc/src/test/resources/TestOrcFile.test1.orc rename to flink-connectors/flink-orc/src/test/resources/test-data-nested.orc diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.listliststructlong.orc b/flink-connectors/flink-orc/src/test/resources/test-data-nestedlist.orc similarity index 100% rename from flink-connectors/flink-orc/src/test/resources/TestOrcFile.listliststructlong.orc rename to flink-connectors/flink-orc/src/test/resources/test-data-nestedlist.orc diff --git a/flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc b/flink-connectors/flink-orc/src/test/resources/test-data-timetypes.orc similarity index 100% rename from flink-connectors/flink-orc/src/test/resources/TestOrcFile.testDate1900.orc rename to flink-connectors/flink-orc/src/test/resources/test-data-timetypes.orc diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java index 1d4b8a151e63c..24ccfb1722957 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java @@ -355,4 +355,21 @@ public TypeComparator createTypeComparator(ExecutionConfig config) { comparatorOrders); } } + + /** + * Creates a {@link RowTypeInfo} with projected fields. + * + * @param rowType The original RowTypeInfo whose fields are projected + * @param fieldMapping The field mapping of the projection + * @return A RowTypeInfo with projected fields. + */ + public static RowTypeInfo projectFields(RowTypeInfo rowType, int[] fieldMapping) { + TypeInformation[] fieldTypes = new TypeInformation[fieldMapping.length]; + String[] fieldNames = new String[fieldMapping.length]; + for (int i = 0; i < fieldMapping.length; i++) { + fieldTypes[i] = rowType.getTypeAt(fieldMapping[i]); + fieldNames[i] = rowType.getFieldNames()[fieldMapping[i]]; + } + return new RowTypeInfo(fieldTypes, fieldNames); + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala index 337025fa88f83..ca8e56921390c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala @@ -29,7 +29,7 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceTable} -import org.apache.flink.table.sources.{TableSource, TableSourceUtil} +import org.apache.flink.table.sources.{FilterableTableSource, TableSource, TableSourceUtil} import scala.collection.JavaConverters._ @@ -62,7 +62,19 @@ class FlinkLogicalTableSourceScan( override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { val rowCnt = metadata.getRowCount(this) - planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * estimateRowSize(getRowType)) + + val adjustedCnt: Double = tableSource match { + case f: FilterableTableSource[_] if f.isFilterPushedDown => + // ensure we prefer FilterableTableSources with pushed-down filters. + rowCnt - 1.0 + case _ => + rowCnt + } + + planner.getCostFactory.makeCost( + adjustedCnt, + adjustedCnt, + adjustedCnt * estimateRowSize(getRowType)) } override def explainTerms(pw: RelWriter): RelWriter = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala index 9c06135e46806..a377a95f4a15f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala @@ -202,6 +202,18 @@ class RexNodeToExpressionConverter( // convert from BigDecimal to Double literal.getValueAs(classOf[java.lang.Double]) + case _@BasicTypeInfo.STRING_TYPE_INFO => + // convert from NlsString to String + literal.getValueAs(classOf[java.lang.String]) + + case _@BasicTypeInfo.BOOLEAN_TYPE_INFO => + // convert to Boolean + literal.getValueAs(classOf[java.lang.Boolean]) + + case _@BasicTypeInfo.BIG_DEC_TYPE_INFO => + // convert to BigDecimal + literal.getValueAs(classOf[java.math.BigDecimal]) + case _ => literal.getValue }