From 44a8841b0e0210f1c96f6fd39fdd6e10e65bf6c7 Mon Sep 17 00:00:00 2001 From: Jungwoo Lee Date: Wed, 25 Jun 2025 20:47:53 +0000 Subject: [PATCH] Implement native ESRI reader Co-authored-by: Dain Sundstrom --- docs/src/main/sphinx/connector/hive.md | 1 + lib/trino-hive-formats/pom.xml | 5 + .../io/trino/hive/formats/HiveClassNames.java | 2 + .../hive/formats/esri/EsriDeserializer.java | 470 ++ .../trino/hive/formats/esri/EsriReader.java | 132 + .../io/trino/hive/formats/esri/OGCType.java | 36 + .../formats/esri/TestEsriDeserializer.java | 637 ++ .../hive/formats/esri/TestEsriReader.java | 283 + .../java/io/trino/plugin/hive/HiveModule.java | 2 + .../trino/plugin/hive/HiveStorageFormat.java | 7 + .../plugin/hive/esri/EsriPageSource.java | 122 + .../hive/esri/EsriPageSourceFactory.java | 113 + .../plugin/hive/BaseHiveConnectorTest.java | 6 + .../io/trino/plugin/hive/TestEsriTable.java | 120 + .../trino/plugin/hive/TestHivePageSink.java | 4 + .../src/test/resources/esri/counties.json | 5142 +++++++++++++++++ .../test/resources/esri/counties_expected.txt | 3 + .../product/hive/TestHiveStorageFormats.java | 2 + 18 files changed, 7087 insertions(+) create mode 100644 lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/EsriDeserializer.java create mode 100644 lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/EsriReader.java create mode 100644 lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/OGCType.java create mode 100644 lib/trino-hive-formats/src/test/java/io/trino/hive/formats/esri/TestEsriDeserializer.java create mode 100644 lib/trino-hive-formats/src/test/java/io/trino/hive/formats/esri/TestEsriReader.java create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/esri/EsriPageSource.java create mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/esri/EsriPageSourceFactory.java create mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestEsriTable.java create mode 100644 plugin/trino-hive/src/test/resources/esri/counties.json create mode 100644 plugin/trino-hive/src/test/resources/esri/counties_expected.txt diff --git a/docs/src/main/sphinx/connector/hive.md b/docs/src/main/sphinx/connector/hive.md index 081c2b1ca29c..239f7dd6696f 100644 --- a/docs/src/main/sphinx/connector/hive.md +++ b/docs/src/main/sphinx/connector/hive.md @@ -53,6 +53,7 @@ In the case of serializable formats, only specific - OPENX_JSON - OpenX JSON SerDe from `org.openx.data.jsonserde.JsonSerDe`. Find more [details about the Trino implementation in the source repository](https://github.com/trinodb/trino/tree/master/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/openxjson/README.md). - TextFile +- ESRI - using `com.esri.hadoop.hive.serde.EsriJsonSerDe` (hive-configuration)= ## General configuration diff --git a/lib/trino-hive-formats/pom.xml b/lib/trino-hive-formats/pom.xml index efdf71c162bf..5f6853c8d9f6 100644 --- a/lib/trino-hive-formats/pom.xml +++ b/lib/trino-hive-formats/pom.xml @@ -17,6 +17,11 @@ + + com.esri.geometry + esri-geometry-api + + com.fasterxml.jackson.core jackson-core diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java index 43a357b59bd9..a663334fea41 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/HiveClassNames.java @@ -45,6 +45,8 @@ public final class HiveClassNames public static final String SEQUENCEFILE_INPUT_FORMAT_CLASS = "org.apache.hadoop.mapred.SequenceFileInputFormat"; public static final String SYMLINK_TEXT_INPUT_FORMAT_CLASS = "org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat"; public static final String TEXT_INPUT_FORMAT_CLASS = "org.apache.hadoop.mapred.TextInputFormat"; + public static final String ESRI_SERDE_CLASS = "com.esri.hadoop.hive.serde.EsriJsonSerDe"; + public static final String ESRI_INPUT_FORMAT_CLASS = "com.esri.json.hadoop.EnclosedEsriJsonInputFormat"; private HiveClassNames() {} } diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/EsriDeserializer.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/EsriDeserializer.java new file mode 100644 index 000000000000..a97b92ac7571 --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/EsriDeserializer.java @@ -0,0 +1,470 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License 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 io.trino.hive.formats.esri; + +import com.esri.core.geometry.Geometry; +import com.esri.core.geometry.GeometryEngine; +import com.esri.core.geometry.MapGeometry; +import com.esri.core.geometry.ogc.OGCGeometry; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.slice.Slices; +import io.trino.hive.formats.line.Column; +import io.trino.plugin.base.type.DecodedTimestamp; +import io.trino.spi.PageBuilder; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DecimalConversions; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Int128; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.lang.invoke.VarHandle; +import java.math.BigDecimal; +import java.nio.ByteOrder; +import java.sql.Date; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeParseException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static com.fasterxml.jackson.core.JsonToken.END_OBJECT; +import static com.fasterxml.jackson.core.JsonToken.FIELD_NAME; +import static com.fasterxml.jackson.core.JsonToken.START_ARRAY; +import static com.fasterxml.jackson.core.JsonToken.START_OBJECT; +import static com.fasterxml.jackson.core.JsonToken.VALUE_NULL; +import static com.fasterxml.jackson.core.JsonToken.VALUE_NUMBER_INT; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.base.type.TrinoTimestampEncoderFactory.createTimestampEncoder; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.Chars.truncateToLengthAndTrimSpaces; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.Decimals.overflows; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_SECOND; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.Varchars.truncateToLength; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.StrictMath.floorDiv; +import static java.lang.StrictMath.floorMod; +import static java.lang.StrictMath.toIntExact; +import static java.lang.String.format; +import static java.math.RoundingMode.HALF_UP; +import static java.time.ZoneOffset.UTC; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public final class EsriDeserializer +{ + private static final VarHandle INT_HANDLE_BIG_ENDIAN = MethodHandles.byteArrayViewVarHandle(int[].class, ByteOrder.BIG_ENDIAN); + private static final String GEOMETRY_FIELD_NAME = "geometry"; + private static final String ATTRIBUTES_FIELD_NAME = "attributes"; + private static final DateTimeFormatter DATE_FORMATTER = + DateTimeFormatter.ofPattern("yyyy-M-d").withZone(UTC); + private static final List TIMESTAMP_FORMATTERS = List.of( + DateTimeFormatter.ofPattern("yyyy-M-d HH:mm:ss.SSS").withZone(UTC), + DateTimeFormatter.ofPattern("yyyy-M-d HH:mm:ss").withZone(UTC), + DateTimeFormatter.ofPattern("yyyy-M-d HH:mm").withZone(UTC), + DATE_FORMATTER); + + private final int geometryColumn; + private final List columns; + private final Map columnIndex; + private final List types; + private final boolean[] fieldWritten; + + public EsriDeserializer(List columns) + { + this.columns = ImmutableList.copyOf(requireNonNull(columns, "columns is null")); + this.types = columns.stream() + .map(Column::type) + .collect(toImmutableList()); + this.fieldWritten = new boolean[columns.size()]; + + for (Column column : columns) { + validateSupportedType(column.type(), column.name()); + } + + ImmutableMap.Builder columnNameBuilder = ImmutableMap.builder(); + for (int index = 0; index < columns.size(); index++) { + columnNameBuilder.put(columns.get(index).name().toLowerCase(ENGLISH), index); + } + columnIndex = columnNameBuilder.buildOrThrow(); + + int geometryColumn = -1; + for (int i = 0; i < columns.size(); i++) { + if (columns.get(i).type() == VARBINARY) { + if (geometryColumn >= 0) { + throw new IllegalArgumentException("Multiple binary columns defined. Define only one binary column for geometries"); + } + geometryColumn = i; + } + } + this.geometryColumn = geometryColumn; + } + + public List getTypes() + { + return types; + } + + public void deserialize(PageBuilder pageBuilder, JsonParser parser) + throws IOException + { + if (parser.currentToken() != START_OBJECT) { + throw invalidJson("start of object expected"); + } + + Arrays.fill(fieldWritten, false); + while (nextObjectField(parser)) { + String fieldName = parser.currentName(); + if (nextTokenRequired(parser) == VALUE_NULL) { + continue; + } + if (GEOMETRY_FIELD_NAME.equals(fieldName)) { + parseGeometry(parser, pageBuilder); + } + else if (ATTRIBUTES_FIELD_NAME.equals(fieldName)) { + parseAttributes(parser, pageBuilder); + } + else { + skipCurrentValue(parser); + } + } + + pageBuilder.declarePosition(); + + for (int i = 0; i < columns.size(); i++) { + if (!fieldWritten[i]) { + pageBuilder.getBlockBuilder(i).appendNull(); + } + } + } + + private BlockBuilder getBlockBuilderForWrite(PageBuilder pageBuilder, int fieldIndex) + { + // if the geometry column is already written, overwrite the last value + BlockBuilder blockBuilder = pageBuilder.getBlockBuilder(fieldIndex); + if (fieldWritten[fieldIndex]) { + blockBuilder.resetTo(blockBuilder.getPositionCount() - 1); + } + fieldWritten[fieldIndex] = true; + return blockBuilder; + } + + private void parseGeometry(JsonParser parser, PageBuilder pageBuilder) + throws IOException + { + if (parser.currentToken() != START_OBJECT) { + throw invalidJson("geometry is not an object"); + } + + // if geometry is not mapped to a column, skip it + if (geometryColumn <= -1) { + skipCurrentValue(parser); + return; + } + + MapGeometry mapGeometry = GeometryEngine.jsonToGeometry(parser); + OGCGeometry ogcGeometry = OGCGeometry.createFromEsriGeometry(mapGeometry.getGeometry(), mapGeometry.getSpatialReference()); + Geometry geometry = ogcGeometry.getEsriGeometry(); + if (geometry == null) { + throw new IllegalArgumentException("Could not parse geometry"); + } + + byte[] shape = GeometryEngine.geometryToEsriShape(geometry); + if (shape == null) { + throw new IllegalArgumentException("Could not serialize geometry shape"); + } + + byte[] shapeHeader = new byte[4 + 1 + shape.length]; + // write the Spatial Reference System Identifier (a.k.a, the well-known ID) + INT_HANDLE_BIG_ENDIAN.set(shapeHeader, 0, ogcGeometry.SRID()); + // write the geometry type + OGCType ogcType = switch (ogcGeometry.geometryType()) { + case "Point" -> OGCType.ST_POINT; + case "LineString" -> OGCType.ST_LINESTRING; + case "Polygon" -> OGCType.ST_POLYGON; + case "MultiPoint" -> OGCType.ST_MULTIPOINT; + case "MultiLineString" -> OGCType.ST_MULTILINESTRING; + case "MultiPolygon" -> OGCType.ST_MULTIPOLYGON; + case null, default -> OGCType.UNKNOWN; + }; + shapeHeader[4] = ogcType.getIndex(); + // write the serialized shape + System.arraycopy(shape, 0, shapeHeader, 5, shape.length); + + // write the shape to the page + VARBINARY.writeSlice(getBlockBuilderForWrite(pageBuilder, geometryColumn), Slices.wrappedBuffer(shapeHeader)); + } + + private void parseAttributes(JsonParser parser, PageBuilder pageBuilder) + throws IOException + { + if (parser.currentToken() != START_OBJECT) { + throw invalidJson("attributes is not an object"); + } + while (nextObjectField(parser)) { + String attributeName = parser.getText().toLowerCase(ENGLISH); + parser.nextToken(); + Integer fieldIndex = columnIndex.get(attributeName); + if (fieldIndex != null) { + Column column = columns.get(fieldIndex); + parseAttribute(parser, column.type(), column.name(), getBlockBuilderForWrite(pageBuilder, fieldIndex)); + } + else { + skipCurrentValue(parser); + } + } + } + + private static void parseAttribute(JsonParser parser, Type columnType, String columnName, BlockBuilder builder) + { + if (VARBINARY.equals(columnType)) { + throw new UnsupportedTypeException(columnType, columnName); + } + + if (parser.getCurrentToken() == VALUE_NULL) { + builder.appendNull(); + return; + } + + try { + if (BOOLEAN.equals(columnType)) { + columnType.writeBoolean(builder, parser.getBooleanValue()); + } + else if (BIGINT.equals(columnType)) { + columnType.writeLong(builder, parser.getLongValue()); + } + else if (INTEGER.equals(columnType)) { + columnType.writeLong(builder, parser.getIntValue()); + } + else if (SMALLINT.equals(columnType)) { + columnType.writeLong(builder, parser.getShortValue()); + } + else if (TINYINT.equals(columnType)) { + columnType.writeLong(builder, parser.getByteValue()); + } + else if (columnType instanceof DecimalType decimalType) { + parseDecimal(parser.getText(), decimalType, builder); + } + else if (REAL.equals(columnType)) { + columnType.writeLong(builder, floatToRawIntBits(parser.getFloatValue())); + } + else if (DOUBLE.equals(columnType)) { + columnType.writeDouble(builder, parser.getDoubleValue()); + } + else if (DATE.equals(columnType)) { + columnType.writeLong(builder, toIntExact(parseDate(parser).getTime() / 86400000L)); + } + else if (columnType instanceof TimestampType timestampType) { + Timestamp timestamp = parseTimestamp(parser); + DecodedTimestamp decodedTimestamp = createDecodedTimestamp(timestamp); + createTimestampEncoder(timestampType, DateTimeZone.UTC).write(decodedTimestamp, builder); + } + else if (columnType instanceof VarcharType varcharType) { + columnType.writeSlice(builder, truncateToLength(Slices.utf8Slice(parser.getText()), varcharType)); + } + else if (columnType instanceof CharType charType) { + columnType.writeSlice(builder, truncateToLengthAndTrimSpaces(Slices.utf8Slice(parser.getText()), charType)); + } + else { + // This should never happen due to prior validation + throw new UnsupportedTypeException(columnType, columnName); + } + } + catch (UnsupportedTypeException e) { + throw e; + } + catch (Exception e) { + // invalid columns are ignored + builder.appendNull(); + } + } + + private static void validateSupportedType(Type type, String columnName) + { + if (BOOLEAN.equals(type) || + BIGINT.equals(type) || + INTEGER.equals(type) || + SMALLINT.equals(type) || + TINYINT.equals(type) || + type instanceof DecimalType || + REAL.equals(type) || + DOUBLE.equals(type) || + DATE.equals(type) || + type instanceof TimestampType || + type instanceof VarcharType || + type instanceof CharType || + VARBINARY.equals(type)) { + return; + } + throw new UnsupportedTypeException(type, columnName); + } + + private static DecodedTimestamp createDecodedTimestamp(Timestamp timestamp) + { + long millis = timestamp.getTime(); + long epochSeconds = floorDiv(millis, (long) MILLISECONDS_PER_SECOND); + long fractionalSecond = floorMod(millis, (long) MILLISECONDS_PER_SECOND); + int nanosOfSecond = toIntExact(fractionalSecond * (long) NANOSECONDS_PER_MILLISECOND); + + return new DecodedTimestamp(epochSeconds, nanosOfSecond); + } + + private static void parseDecimal(String value, DecimalType decimalType, BlockBuilder builder) + { + BigDecimal bigDecimal; + try { + bigDecimal = new BigDecimal(value).setScale(DecimalConversions.intScale(decimalType.getScale()), HALF_UP); + } + catch (NumberFormatException e) { + throw new NumberFormatException(format("Cannot convert '%s' to %s. Value is not a number.", value, decimalType)); + } + + if (overflows(bigDecimal, decimalType.getPrecision())) { + throw new IllegalArgumentException(format("Cannot convert '%s' to %s. Value too large.", value, decimalType)); + } + + if (decimalType.isShort()) { + decimalType.writeLong(builder, bigDecimal.unscaledValue().longValueExact()); + } + else { + decimalType.writeObject(builder, Int128.valueOf(bigDecimal.unscaledValue())); + } + } + + // NOTE: It only supports UTC timezone. + private static Date parseDate(JsonParser parser) + throws IOException + { + if (VALUE_NUMBER_INT == parser.getCurrentToken()) { + long epoch = parser.getLongValue(); + return new Date(epoch); + } + + try { + LocalDate localDate = LocalDate.parse(parser.getText(), DATE_FORMATTER); + // Add 12 hours (43200000L milliseconds) to handle noon conversion + return new Date(localDate.atStartOfDay(UTC) + .toInstant() + .toEpochMilli() + 43200000L); + } + catch (DateTimeParseException e) { + throw new IllegalArgumentException(format("Value '%s' cannot be parsed to a Date. Expected format: yyyy-MM-dd", parser.getText()), e); + } + } + + private static Timestamp parseTimestamp(JsonParser parser) + throws IOException + { + if (VALUE_NUMBER_INT == parser.getCurrentToken()) { + long epoch = parser.getLongValue(); + return new Timestamp(epoch); + } + + String value = parser.getText(); + int point = value.indexOf('.'); + String dateStr = point < 0 ? value : value.substring(0, Math.min(point + 4, value.length())); + + for (DateTimeFormatter formatter : TIMESTAMP_FORMATTERS) { + try { + Instant instant; + if (formatter.equals(DATE_FORMATTER)) { + LocalDate localDate = LocalDate.parse(dateStr, formatter); + instant = localDate.atStartOfDay(UTC).toInstant(); + } + else { + LocalDateTime dateTime = LocalDateTime.parse(dateStr, formatter); + instant = dateTime.atZone(UTC).toInstant(); + } + return Timestamp.from(instant); + } + catch (DateTimeParseException e) { + // Try next formatter + } + } + + throw new IllegalArgumentException(format("Value '%s' cannot be parsed to a Timestamp", dateStr)); + } + + static boolean nextObjectField(JsonParser parser) + throws IOException + { + JsonToken token = nextTokenRequired(parser); + if (token == FIELD_NAME) { + return true; + } + if (token == END_OBJECT) { + return false; + } + throw invalidJson("field name expected, but was " + token); + } + + static JsonToken nextTokenRequired(JsonParser parser) + throws IOException + { + JsonToken token = parser.nextToken(); + if (token == null) { + throw invalidJson("object is truncated"); + } + return token; + } + + static void skipCurrentValue(JsonParser parser) + throws IOException + { + JsonToken valueToken = parser.currentToken(); + if ((valueToken == START_ARRAY) || (valueToken == START_OBJECT)) { + // if the current token is a beginning of an array or object, move the stream forward + // skipping any child tokens till we're at the corresponding END_ARRAY or END_OBJECT token + parser.skipChildren(); + } + // At the end of this function, the stream should be pointing to the last token that + // corresponds to the value being skipped. This way, the next call to nextToken + // will advance it to the next field name. + } + + static IOException invalidJson(String message) + { + return new IOException("Invalid JSON: " + message); + } + + public static class UnsupportedTypeException + extends RuntimeException + { + public UnsupportedTypeException(Type columnType, String columnName) + { + super("Column '" + columnName + "' with type: " + columnType.getDisplayName() + " is not supported"); + } + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/EsriReader.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/EsriReader.java new file mode 100644 index 000000000000..e2fb5272280d --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/EsriReader.java @@ -0,0 +1,132 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License 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 io.trino.hive.formats.esri; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.google.common.io.Closer; +import com.google.common.io.CountingInputStream; +import io.trino.spi.PageBuilder; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; + +import static com.fasterxml.jackson.core.JsonFactory.Feature.INTERN_FIELD_NAMES; +import static io.trino.hive.formats.esri.EsriDeserializer.invalidJson; +import static io.trino.hive.formats.esri.EsriDeserializer.nextObjectField; +import static io.trino.hive.formats.esri.EsriDeserializer.nextTokenRequired; +import static io.trino.hive.formats.esri.EsriDeserializer.skipCurrentValue; +import static io.trino.plugin.base.util.JsonUtils.jsonFactoryBuilder; +import static java.util.Objects.requireNonNull; + +public final class EsriReader + implements Closeable +{ + private static final JsonFactory JSON_FACTORY = jsonFactoryBuilder() + .disable(INTERN_FIELD_NAMES) + .build(); + private static final String FEATURES_NAME = "features"; + + private final CountingInputStream inputStream; + private final EsriDeserializer esriDeserializer; + private final JsonParser parser; + + private long readTimeNanos; + private boolean closed; + + public EsriReader(InputStream inputStream, EsriDeserializer esriDeserializer) + throws IOException + { + this.inputStream = new CountingInputStream(requireNonNull(inputStream, "inputStream is null")); + this.esriDeserializer = requireNonNull(esriDeserializer, "esriDeserializer is null"); + + parser = JSON_FACTORY.createParser(this.inputStream); + if (nextTokenRequired(parser) != JsonToken.START_OBJECT) { + throw invalidJson("File must start with a JSON object"); + } + + // Advance to the features field + while (nextObjectField(parser)) { + String fieldName = parser.currentName(); + JsonToken fieldValue = nextTokenRequired(parser); + if (FEATURES_NAME.equals(fieldName)) { + // read the array start token + if (fieldValue == JsonToken.VALUE_NULL) { + close(); + return; + } + if (fieldValue != JsonToken.START_ARRAY) { + throw invalidJson("Features field must be an array"); + } + break; + } + else { + // skip the field value + skipCurrentValue(parser); + } + } + } + + @Override + public void close() + throws IOException + { + closed = true; + try (Closer closer = Closer.create()) { + closer.register(inputStream); + closer.register(parser); + } + } + + public boolean next(PageBuilder pageBuilder) + throws IOException + { + if (closed) { + return false; + } + + long start = System.nanoTime(); + try { + JsonToken token = parser.nextToken(); + if (token == null || token == JsonToken.END_ARRAY) { + // everything after the features array is ignored + close(); + return false; + } + esriDeserializer.deserialize(pageBuilder, parser); + return true; + } + finally { + long duration = System.nanoTime() - start; + readTimeNanos += duration; + } + } + + public long getBytesRead() + { + return inputStream.getCount(); + } + + public long getReadTimeNanos() + { + return readTimeNanos; + } + + public boolean isClosed() + { + return closed; + } +} diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/OGCType.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/OGCType.java new file mode 100644 index 000000000000..b9fcffe0b17c --- /dev/null +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/esri/OGCType.java @@ -0,0 +1,36 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License 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 io.trino.hive.formats.esri; + +public enum OGCType { + UNKNOWN((byte) 0), + ST_POINT((byte) 1), + ST_LINESTRING((byte) 2), + ST_POLYGON((byte) 3), + ST_MULTIPOINT((byte) 4), + ST_MULTILINESTRING((byte) 5), + ST_MULTIPOLYGON((byte) 6); + + private final byte index; + + OGCType(byte index) + { + this.index = index; + } + + public byte getIndex() + { + return this.index; + } +} diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/esri/TestEsriDeserializer.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/esri/TestEsriDeserializer.java new file mode 100644 index 000000000000..34dbf3e93866 --- /dev/null +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/esri/TestEsriDeserializer.java @@ -0,0 +1,637 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License 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 io.trino.hive.formats.esri; + +import com.esri.core.geometry.Geometry; +import com.esri.core.geometry.GeometryEngine; +import com.esri.core.geometry.Point; +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.JsonParser; +import com.google.common.collect.ImmutableList; +import io.trino.hive.formats.line.Column; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.block.Block; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DecimalType; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.List; + +import static com.fasterxml.jackson.core.JsonToken.START_OBJECT; +import static io.trino.plugin.base.util.JsonUtils.jsonFactory; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.Float.floatToRawIntBits; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestEsriDeserializer +{ + private static final JsonFactory JSON_FACTORY = jsonFactory(); + private static final List COLUMNS = ImmutableList.of( + new Column("id", BIGINT, 0), + new Column("name", VARCHAR, 1), + new Column("active", BOOLEAN, 2), + new Column("value", DOUBLE, 3), + new Column("date", DATE, 4), + new Column("timestamp", TIMESTAMP_MILLIS, 5), + new Column("geometry", VARBINARY, 6), + new Column("count", INTEGER, 7), + new Column("price", DecimalType.createDecimalType(10, 2), 8), + new Column("small_num", SMALLINT, 9), + new Column("tiny_num", TINYINT, 10), + new Column("real_num", REAL, 11), + new Column("fixed_text", CharType.createCharType(10), 12)); + + @Test + public void testDeserializeSimpleFeature() + throws IOException + { + String json = """ + { + "attributes": { + "id": 1, + "name": "Test Feature", + "active": true, + "value": 123.45, + "date": 1741034025839, + "timestamp": 1741034025839, + "count": 42, + "price": "1234.56" + }, + "geometry": { + "x": 10, + "y": 20 + } + } + """; + + Page page = parse(json); + assertThat(page.getPositionCount()).isEqualTo(1); + + for (int i = 0; i < 9; i++) { + assertThat(page.getBlock(i).isNull(0)) + .as("Column at index " + i + " should not be null") + .isFalse(); + } + assertThat(BIGINT.getLong(page.getBlock(0), 0)).isEqualTo(1L); + assertThat(VARCHAR.getSlice(page.getBlock(1), 0).toStringUtf8()).isEqualTo("Test Feature"); + assertThat(BOOLEAN.getBoolean(page.getBlock(2), 0)).isTrue(); + assertThat(DOUBLE.getDouble(page.getBlock(3), 0)).isEqualTo(123.45); + assertThat(DATE.getLong(page.getBlock(4), 0)).isEqualTo(20150); + assertThat(TIMESTAMP_MILLIS.getLong(page.getBlock(5), 0)).isEqualTo(1741034025839000L); + assertGeometry(page, new Point(10, 20)); + assertThat(INTEGER.getLong(page.getBlock(7), 0)).isEqualTo(42); + + DecimalType decimalType = DecimalType.createDecimalType(10, 2); + Block decimalBlock = page.getBlock(8); + assertThat(decimalType.getLong(decimalBlock, 0)).isEqualTo(123456L); + } + + @Test + public void testDeserializeNullValues() throws IOException + { + String json = """ + { + "attributes": { + "id": null, + "name": null, + "active": null, + "value": null, + "date": null, + "timestamp": null, + "count": null, + "price": null + }, + "geometry": null + } + """; + + Page page = parse(json); + for (int i = 0; i < 9; i++) { + assertThat(page.getBlock(i).isNull(0)) + .as("Column at index " + i + " should be null") + .isTrue(); + } + } + + @Test + public void testSupportedAttributeTypes() + throws IOException + { + String json = """ + { + "attributes": { + "id": 9223372036854775807, + "name": "string value", + "active": true, + "value": 123.456789, + "date": "2025-03-03", + "timestamp": "2025-03-03 12:34:56.789", + "count": 2147483647, + "price": "99999999.99", + "small_num": 32767, + "tiny_num": 127, + "real_num": 3.14159, + "fixed_text": "FIXED " + } + } + """; + + Page page = parse(json); + + // Test BIGINT + assertThat(BIGINT.getLong(page.getBlock(0), 0)).isEqualTo(9223372036854775807L); + + // Test VARCHAR + assertThat(VARCHAR.getSlice(page.getBlock(1), 0).toStringUtf8()).isEqualTo("string value"); + + // Test BOOLEAN + assertThat(BOOLEAN.getBoolean(page.getBlock(2), 0)).isTrue(); + + // Test DOUBLE + assertThat(DOUBLE.getDouble(page.getBlock(3), 0)).isEqualTo(123.456789); + + // Test DATE + assertThat(DATE.getLong(page.getBlock(4), 0)).isEqualTo(20150); + + // Test TIMESTAMP + assertThat(TIMESTAMP_MILLIS.getLong(page.getBlock(5), 0)).isEqualTo(1741005296789000L); + + // Test INTEGER + assertThat(INTEGER.getLong(page.getBlock(7), 0)).isEqualTo(2147483647); + + // Test DECIMAL + DecimalType decimalType = DecimalType.createDecimalType(10, 2); + Block decimalBlock = page.getBlock(8); + assertThat(decimalType.getLong(decimalBlock, 0)).isEqualTo(9999999999L); + + // Test SMALLINT + assertThat(SMALLINT.getLong(page.getBlock(9), 0)).isEqualTo(32767); + + // Test TINYINT + assertThat(TINYINT.getLong(page.getBlock(10), 0)).isEqualTo(127); + + // Test REAL + float expectedFloat = 3.14159f; + assertThat(REAL.getLong(page.getBlock(11), 0)).isEqualTo(floatToRawIntBits(expectedFloat)); + + // Test CHAR + CharType charType = CharType.createCharType(10); + assertThat(charType.getSlice(page.getBlock(12), 0).toStringUtf8()).isEqualTo("FIXED"); + } + + @Test + public void testUnsupportedAttributeTypes() + { + String json = """ + { + "attributes": { + "id": 1, + "name": "Test Feature", + "varbinary_field": "Some binary data", + "value": 123.45 + }, + "geometry": { + "x": 10, + "y": 20 + } + } + """; + + List columns = ImmutableList.of( + new Column("id", BIGINT, 0), + new Column("name", VARCHAR, 1), + new Column("varbinary_field", VARBINARY, 2), + new Column("value", DOUBLE, 3) + ); + + assertThatThrownBy(() -> parse(json, columns)) + .isInstanceOf(EsriDeserializer.UnsupportedTypeException.class) + .hasMessageContaining("Column 'varbinary_field' with type: varbinary is not supported"); + } + + @Test + public void testDeserializeMissingColumns() throws IOException + { + String json = """ + { + "attributes": { + }, + "geometry": null + } + """; + + Page page = parse(json); + for (int i = 0; i < 9; i++) { + assertThat(page.getBlock(i).isNull(0)) + .as("Column at index " + i + " should be null") + .isTrue(); + } + } + + @Test + public void testDeserializeInvalidJson() + { + String invalidJson = "{ This is not valid JSON }"; + assertThatThrownBy(() -> parse(invalidJson)) + .isInstanceOf(JsonParseException.class); + } + + @Test + public void testDeserializeDateFormats() throws IOException { + // Test valid epoch milliseconds (as number) + String jsonEpoch = """ + { + "attributes": { + "date": 1741034025839 + }, + "geometry": null + } + """; + + Page page = parse(jsonEpoch); + assertThat(DATE.getLong(page.getBlock(4), 0)).isEqualTo(20150); + + // Test valid ISO date formats + String[] validIsoFormats = { + "2025-3-3", // without leading zeros + "2025-03-03" // with leading zeros + }; + + for (String dateStr : validIsoFormats) { + String json = String.format(""" + { + "attributes": { + "date": "%s" + }, + "geometry": null + } + """, dateStr); + + page = parse(json); + assertThat(DATE.getLong(page.getBlock(4), 0)).isEqualTo(20150); + } + + // Test invalid date formats - should result in null + String[] invalidDateFormats = { + "2025/03/03", // slash-separated date + "03/03/2025" // US-style date + }; + + for (String dateStr : invalidDateFormats) { + String json = String.format(""" + { + "attributes": { + "date": "%s" + }, + "geometry": null + } + """, dateStr); + + page = parse(json); + assertThat(page.getBlock(4).isNull(0)) + .as("Date '%s' should be parsed as null", dateStr) + .isTrue(); + } + } + + @Test + public void testDeserializeTimestampFormats() throws IOException { + String[] validTimestampFormats = { + "2025-03-03 00:00:00.000", // with leading zeros + "2025-03-03 00:00:00", + "2025-03-03 00:00", + "2025-03-03", + "2025-3-3 00:00:00.000", // without leading zeros + "2025-3-3 00:00:00", + "2025-3-3 00:00", + "2025-3-3" + }; + + String[] invalidTimestampFormats = { + "2025/03/03 00:00:00", // slash-separated date with time + "03/03/2025 00:00:00 AM", // US-style date with time + "2025-03-03T00:00:00.000Z" // ISO 8601 format + }; + + // Test valid epoch milliseconds (as number) + String jsonEpoch = """ + { + "attributes": { + "timestamp": 1741034025839 + }, + "geometry": null + } + """; + + Page page = parse(jsonEpoch); + assertThat(TIMESTAMP_MILLIS.getLong(page.getBlock(5), 0)) + .isEqualTo(1741034025839000L); + + // Test valid timestamps + for (String timestampStr : validTimestampFormats) { + String json = String.format(""" + { + "attributes": { + "timestamp": "%s" + }, + "geometry": null + } + """, timestampStr); + + page = parse(json); + assertThat(TIMESTAMP_MILLIS.getLong(page.getBlock(5), 0)) + .isEqualTo(1740960000000000L); + } + + // Test invalid timestamps + for (String timestampStr : invalidTimestampFormats) { + String json = String.format(""" + { + "attributes": { + "timestamp": "%s" + }, + "geometry": null + } + """, timestampStr); + + page = parse(json); + assertThat(page.getBlock(5).isNull(0)) + .as("Timestamp '%s' should be parsed as null", timestampStr) + .isTrue(); + } + } + + @Test + public void testDeserializeInvalidDateFormat() throws IOException { + String json = """ + { + "attributes": { + "date": "invalid-date" + }, + "geometry": null + } + """; + + Page page = parse(json); + assertThat(page.getBlock(4).isNull(0)).isTrue(); + } + + @Test + public void testDeserializeInvalidTimestampFormat() throws IOException { + String json = """ + { + "attributes": { + "timestamp": "invalid-timestamp" + }, + "geometry": null + } + """; + + Page page = parse(json); + assertThat(page.getBlock(5).isNull(0)).isTrue(); + } + + @Test + public void testDeserializeDateOutOfRange() throws IOException { + String json = """ + { + "attributes": { + "date": "9999-12-31" + }, + "geometry": null + } + """; + + Page page = parse(json); + assertThat(page.getBlock(4).isNull(0)).isFalse(); + assertThat(DATE.getLong(page.getBlock(4), 0)).isEqualTo(2932896); + } + + @Test + public void testDeserializeInvalid() + throws IOException + { + String json = """ + { + "extra-junk": { + "geometry": null, + "attributes": { + "id": 42 + } + }, + "attributes": { + "id": 1 + }, + "geometry": { + "x": 10, + "y": 20 + } + } + """; + + Page page = parse(json); + assertThat(BIGINT.getLong(page.getBlock(0), 0)).isEqualTo(1L); + assertThat(page.getBlock(6).isNull(0)).isFalse(); + assertThat(VARBINARY.getSlice(page.getBlock(6), 0)).isNotNull(); + } + + @Test + public void testMissingAttributes() + throws IOException + { + String json = """ + { + "geometry": { + "x": 10, + "y": 20 + } + } + """; + + Page page = parse(json); + assertGeometry(page, new Point(10, 20)); + } + + @Test + public void testDuplicateGeometry() + throws IOException + { + String json = """ + { + "geometry": { + "x": 88, + "y": 99 + }, + "geometry": { + "x": 10, + "y": 20 + } + } + """; + + Page page = parse(json); + assertGeometry(page, new Point(10, 20)); + } + + @Test + public void testArrayGeometryFails() + { + String json = """ + { + "geometry": [] + } + """; + + assertThatThrownBy(() -> parse(json)) + .isInstanceOf(IOException.class) + .hasMessage("Invalid JSON: geometry is not an object"); + } + + @Test + public void testNumberGeometryFails() + { + String json = """ + { + "geometry": 42 + } + """; + + assertThatThrownBy(() -> parse(json)) + .isInstanceOf(IOException.class) + .hasMessage("Invalid JSON: geometry is not an object"); + } + + @Test + public void testNullAttributes() + throws IOException + { + String json = """ + { + "geometry": { + "x": 5, + "y": 7 + } + } + """; + + Page page = parse(json); + assertGeometry(page, new Point(5, 7)); + } + + @Test + public void testArrayAttributes() + { + String json = """ + { + "attributes": [] + } + """; + + assertThatThrownBy(() -> parse(json)) + .isInstanceOf(IOException.class) + .hasMessage("Invalid JSON: attributes is not an object"); + } + + @Test + public void testNumberAttributes() + { + String json = """ + { + "attributes": 42 + } + """; + + assertThatThrownBy(() -> parse(json)) + .isInstanceOf(IOException.class) + .hasMessage("Invalid JSON: attributes is not an object"); + } + + @Test + public void testDuplicateAttribute() + throws IOException + { + String json = """ + { + "attributes": { + "id": 1, + "id": 2 + } + } + """; + + Page page = parse(json); + assertThat(BIGINT.getLong(page.getBlock(0), 0)).isEqualTo(2L); + } + + private static Page parse(String json) + throws IOException + { + return parse(json, COLUMNS); + } + + private static Page parse(String json, List columns) + throws IOException { + JsonParser jsonParser = JSON_FACTORY.createParser(json); + assertThat(jsonParser.nextToken()).isEqualTo(START_OBJECT); + + EsriDeserializer deserializer = new EsriDeserializer(columns); + PageBuilder pageBuilder = new PageBuilder(deserializer.getTypes()); + deserializer.deserialize(pageBuilder, jsonParser); + Page page = pageBuilder.build(); + assertThat(page.getPositionCount()).isEqualTo(1); + return page; + } + + private static void assertGeometry(Page page, Geometry expected) + { + if (expected == null) { + assertThat(page.getBlock(6).isNull(0)).isTrue(); + return; + } + + assertThat(page.getBlock(6).isNull(0)).isFalse(); + + byte[] actual = VARBINARY.getSlice(page.getBlock(6), 0).getBytes(); + + byte[] expectedShape = GeometryEngine.geometryToEsriShape(expected); + byte[] expectedBytes = new byte[4 + 1 + expectedShape.length]; + + OGCType ogcType = switch (expected.getType()) { + case Point -> OGCType.ST_POINT; + case Line -> OGCType.ST_LINESTRING; + case Polygon -> OGCType.ST_POLYGON; + case MultiPoint -> OGCType.ST_MULTIPOINT; + case Polyline -> OGCType.ST_MULTILINESTRING; + default -> OGCType.UNKNOWN; + }; + expectedBytes[4] = ogcType.getIndex(); + System.arraycopy(expectedShape, 0, expectedBytes, 5, expectedShape.length); + + assertThat(actual).isEqualTo(expectedBytes); + } +} diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/esri/TestEsriReader.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/esri/TestEsriReader.java new file mode 100644 index 000000000000..492a58bd67ec --- /dev/null +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/esri/TestEsriReader.java @@ -0,0 +1,283 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License 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 io.trino.hive.formats.esri; + +import io.trino.hive.formats.line.Column; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.List; + +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestEsriReader +{ + private static final List TEST_COLUMNS = List.of( + new Column("id", BIGINT, 0), + new Column("name", VARCHAR, 1), + new Column("geometry", VARBINARY, 2)); + + @Test + public void testReadSimpleFeatures() + throws IOException + { + String json = """ + { + "features": [ + { + "attributes": { + "id": 1, + "name": "Feature 1" + }, + "geometry": { + "x": 10, + "y": 20 + } + }, + { + "attributes": { + "id": 2, + "name": "Feature 2" + }, + "geometry": { + "x": 30, + "y": 40 + } + } + ] + } + """; + + Page page = readAll(json); + assertThat(page.getPositionCount()).isEqualTo(2); + } + + @Test + public void testEmptyFeatures() + throws IOException + { + String json = """ + { + "features": [] + } + """; + + Page page = readAll(json); + assertThat(page.getPositionCount()).isZero(); + } + + @Test + public void testNullFeatures() + throws IOException + { + String json = """ + { + "features": null + } + """; + + Page page = readAll(json); + assertThat(page.getPositionCount()).isZero(); + } + + @Test + public void testNumberFeaturesFails() + { + String json = """ + { + "features": 42 + } + """; + + assertThatThrownBy(() -> new EsriReader(new ByteArrayInputStream(json.getBytes(UTF_8)), new EsriDeserializer(TEST_COLUMNS))) + .isInstanceOf(IOException.class) + .hasMessage("Invalid JSON: Features field must be an array"); + } + + @Test + public void testObjectFeaturesFails() + { + String json = """ + { + "features": {} + } + """; + + assertThatThrownBy(() -> new EsriReader(new ByteArrayInputStream(json.getBytes(UTF_8)), new EsriDeserializer(TEST_COLUMNS))) + .isInstanceOf(IOException.class) + .hasMessage("Invalid JSON: Features field must be an array"); + } + + @Test + public void testNoFeaturesArray() + throws IOException + { + String json = """ + { + "someOtherField": [] + } + """; + + Page page = readAll(json); + assertThat(page.getPositionCount()).isZero(); + } + + @Test + public void testLargeRead() + throws IOException + { + StringBuilder jsonBuilder = new StringBuilder(); + jsonBuilder.append("{\"features\":["); + for (int i = 0; i < 1000; i++) { + if (i > 0) { + jsonBuilder.append(","); + } + jsonBuilder.append(String.format( + """ + { + "attributes": { + "id": %d, + "name": "Feature %d" + }, + "geometry": { + "x": %d, + "y": %d + } + } + """, i, i, i * 10, i * 20)); + } + jsonBuilder.append("]}"); + + EsriDeserializer deserializer = new EsriDeserializer(TEST_COLUMNS); + PageBuilder pageBuilder = new PageBuilder(deserializer.getTypes()); + + try (EsriReader reader = new EsriReader(new ByteArrayInputStream(jsonBuilder.toString().getBytes(UTF_8)), deserializer)) { + int count = 0; + while (reader.next(pageBuilder)) { + count++; + } + assertThat(count).isEqualTo(1000); + } + } + + @Test + public void testTruncatedFeaturesAllowed() + throws IOException + { + String json = """ + { + "features": [ + { + "attributes": { + "id": 1 + } + } + ] + EVERYTHING AFTER ARRAY CLOSE IS IGNORED + """; + + Page page = readAll(json); + assertThat(page.getPositionCount()).isEqualTo(1); + } + + @Test + public void testDuplicateFeaturesIgnored() + throws IOException + { + String json = """ + { + "features": [ + { + "attributes": { + "id": 1 + } + } + ], + "features": [ + { + "attributes": { + "id": 2 + } + } + ] + } + """; + + Page page = readAll(json); + assertThat(page.getPositionCount()).isEqualTo(1); + } + + @Test + public void testNestedFeaturesIgnored() + throws IOException + { + String json = """ + { + "bad": { + "features": [ + { + "attributes": { + "id": 1 + } + } + ] + }, + "features": [ + { + "attributes": { + "id": 77 + } + } + ] + } + """; + + Page page = readAll(json); + assertThat(page.getPositionCount()).isEqualTo(1); + assertThat(BIGINT.getLong(page.getBlock(0), 0)).isEqualTo(77); + } + + private static Page readAll(String json) + throws IOException + { + EsriDeserializer deserializer = new EsriDeserializer(TEST_COLUMNS); + EsriReader reader = new EsriReader(new ByteArrayInputStream(json.getBytes(UTF_8)), deserializer); + PageBuilder pageBuilder = new PageBuilder(deserializer.getTypes()); + try { + boolean closed = reader.isClosed(); + while (reader.next(pageBuilder)) { + assertThat(closed).isFalse(); + closed = reader.isClosed(); + } + } + finally { + reader.close(); + assertThat(reader.getBytesRead()).isPositive(); + if (!pageBuilder.isEmpty()) { + assertThat(reader.getReadTimeNanos()).isPositive(); + } + assertThat(reader.isClosed()).isTrue(); + } + + return pageBuilder.build(); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java index d0cc71da8339..afc9ce3fc0e8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java @@ -22,6 +22,7 @@ import io.trino.plugin.base.metrics.FileFormatDataSourceStats; import io.trino.plugin.hive.avro.AvroFileWriterFactory; import io.trino.plugin.hive.avro.AvroPageSourceFactory; +import io.trino.plugin.hive.esri.EsriPageSourceFactory; import io.trino.plugin.hive.fs.CachingDirectoryLister; import io.trino.plugin.hive.fs.DirectoryLister; import io.trino.plugin.hive.fs.TransactionScopeCachingDirectoryListerFactory; @@ -104,6 +105,7 @@ public void configure(Binder binder) pageSourceFactoryBinder.addBinding().to(CsvPageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(JsonPageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(OpenXJsonPageSourceFactory.class).in(Scopes.SINGLETON); + pageSourceFactoryBinder.addBinding().to(EsriPageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(RegexPageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(SimpleTextFilePageSourceFactory.class).in(Scopes.SINGLETON); pageSourceFactoryBinder.addBinding().to(SimpleSequenceFilePageSourceFactory.class).in(Scopes.SINGLETON); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java index b9036479cff0..59cd395fba64 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveStorageFormat.java @@ -33,6 +33,8 @@ import static io.trino.hive.formats.HiveClassNames.AVRO_CONTAINER_OUTPUT_FORMAT_CLASS; import static io.trino.hive.formats.HiveClassNames.AVRO_SERDE_CLASS; import static io.trino.hive.formats.HiveClassNames.COLUMNAR_SERDE_CLASS; +import static io.trino.hive.formats.HiveClassNames.ESRI_INPUT_FORMAT_CLASS; +import static io.trino.hive.formats.HiveClassNames.ESRI_SERDE_CLASS; import static io.trino.hive.formats.HiveClassNames.HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS; import static io.trino.hive.formats.HiveClassNames.HIVE_SEQUENCEFILE_OUTPUT_FORMAT_CLASS; import static io.trino.hive.formats.HiveClassNames.JSON_SERDE_CLASS; @@ -101,6 +103,10 @@ public enum HiveStorageFormat REGEX( REGEX_SERDE_CLASS, TEXT_INPUT_FORMAT_CLASS, + HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS), + ESRI( + ESRI_SERDE_CLASS, + ESRI_INPUT_FORMAT_CLASS, HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS); private final String serde; @@ -135,6 +141,7 @@ public boolean isSplittable(String path) return switch (this) { case ORC, PARQUET, AVRO, RCBINARY, RCTEXT, SEQUENCEFILE -> true; case JSON, OPENX_JSON, TEXTFILE, CSV, REGEX -> CompressionKind.forFile(path).isEmpty(); + case ESRI -> false; }; } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/esri/EsriPageSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/esri/EsriPageSource.java new file mode 100644 index 000000000000..f61d233aed92 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/esri/EsriPageSource.java @@ -0,0 +1,122 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License 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 io.trino.plugin.hive.esri; + +import io.trino.filesystem.Location; +import io.trino.hive.formats.esri.EsriReader; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.SourcePage; + +import java.io.IOException; +import java.util.List; +import java.util.OptionalLong; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.airlift.slice.SizeOf.instanceSize; +import static io.trino.plugin.base.util.Closables.closeAllSuppress; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_CURSOR_ERROR; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class EsriPageSource + implements ConnectorPageSource +{ + private static final int INSTANCE_SIZE = instanceSize(EsriPageSource.class); + + private final EsriReader esriReader; + private final Location filePath; + + private final PageBuilder pageBuilder; + private long completedPositions; + private boolean finished; + + public EsriPageSource(EsriReader esriReader, List columns, Location filePath) + { + this.esriReader = requireNonNull(esriReader, "esriReader is null"); + this.filePath = requireNonNull(filePath, "filePath is null"); + requireNonNull(columns, "columns is null"); + + pageBuilder = new PageBuilder(columns.stream() + .map(HiveColumnHandle::getType) + .collect(toImmutableList())); + } + + @Override + public long getCompletedBytes() + { + return esriReader.getBytesRead(); + } + + @Override + public OptionalLong getCompletedPositions() + { + return OptionalLong.of(completedPositions); + } + + @Override + public long getReadTimeNanos() + { + return esriReader.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return finished; + } + + @Override + public SourcePage getNextSourcePage() + { + try { + while (!pageBuilder.isFull()) { + if (!esriReader.next(pageBuilder)) { + finished = true; + break; + } + } + + Page page = pageBuilder.build(); + completedPositions += page.getPositionCount(); + pageBuilder.reset(); + + return SourcePage.create(page); + } + catch (TrinoException e) { + closeAllSuppress(e, this); + throw e; + } + catch (IOException | RuntimeException e) { + closeAllSuppress(e, this); + throw new TrinoException(HIVE_CURSOR_ERROR, format("Failed to read file at %s", filePath), e); + } + } + + @Override + public long getMemoryUsage() + { + return INSTANCE_SIZE + pageBuilder.getRetainedSizeInBytes(); + } + + @Override + public void close() + throws IOException + { + esriReader.close(); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/esri/EsriPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/esri/EsriPageSourceFactory.java new file mode 100644 index 000000000000..19013af43fa9 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/esri/EsriPageSourceFactory.java @@ -0,0 +1,113 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License 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 io.trino.plugin.hive.esri; + +import com.google.inject.Inject; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoInputFile; +import io.trino.hive.formats.esri.EsriDeserializer; +import io.trino.hive.formats.esri.EsriReader; +import io.trino.hive.formats.line.Column; +import io.trino.plugin.hive.AcidInfo; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HivePageSourceFactory; +import io.trino.plugin.hive.Schema; +import io.trino.plugin.hive.acid.AcidTransaction; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.EmptyPageSource; +import io.trino.spi.predicate.TupleDomain; + +import java.io.InputStream; +import java.util.List; +import java.util.Optional; +import java.util.OptionalInt; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.hive.formats.HiveClassNames.ESRI_INPUT_FORMAT_CLASS; +import static io.trino.hive.formats.HiveClassNames.ESRI_SERDE_CLASS; +import static io.trino.hive.thrift.metastore.hive_metastoreConstants.FILE_INPUT_FORMAT; +import static io.trino.plugin.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT; +import static io.trino.plugin.hive.util.HiveUtil.splitError; +import static java.util.Objects.requireNonNull; + +public class EsriPageSourceFactory + implements HivePageSourceFactory +{ + private final TrinoFileSystemFactory trinoFileSystemFactory; + + @Inject + public EsriPageSourceFactory(TrinoFileSystemFactory trinoFileSystemFactory) + { + this.trinoFileSystemFactory = requireNonNull(trinoFileSystemFactory, "trinoFileSystemFactory is null"); + } + + @Override + public Optional createPageSource( + ConnectorSession session, + Location path, + long start, + long length, + long estimatedFileSize, + long fileModifiedTime, + Schema schema, + List columns, + TupleDomain effectivePredicate, + Optional acidInfo, + OptionalInt bucketNumber, + boolean originalFile, + AcidTransaction transaction) + { + if (!ESRI_SERDE_CLASS.equals(schema.serializationLibraryName()) + || !ESRI_INPUT_FORMAT_CLASS.equals(schema.serdeProperties().get(FILE_INPUT_FORMAT))) { + return Optional.empty(); + } + + checkArgument(acidInfo.isEmpty(), "Acid is not supported for Esri files"); + + // Skip empty inputs + if (length == 0) { + return Optional.of(new EmptyPageSource()); + } + + if (start != 0) { + throw new TrinoException(HIVE_CANNOT_OPEN_SPLIT, "Split start must be 0 for Esri files"); + } + + TrinoFileSystem trinoFileSystem = trinoFileSystemFactory.create(session); + TrinoInputFile inputFile = trinoFileSystem.newInputFile(path); + + // TODO: optimization for small files that should just be read into memory. Consider it later. + try { + InputStream inputStream = inputFile.newStream(); + + List decoderColumns = columns.stream() + .map(hc -> new Column(hc.getName(), hc.getType(), hc.getBaseHiveColumnIndex())) + .collect(toImmutableList()); + + EsriDeserializer esriDeserializer = new EsriDeserializer(decoderColumns); + EsriReader esriReader = new EsriReader(inputStream, esriDeserializer); + EsriPageSource pageSource = new EsriPageSource(esriReader, columns, path); + + return Optional.of(pageSource); + } + catch (Exception e) { + throw new TrinoException(HIVE_CANNOT_OPEN_SPLIT, splitError(e, path, start, length), e); + } + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index 1ddcb39a1b94..405a683b25a7 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -142,6 +142,7 @@ import static io.trino.plugin.hive.HiveQueryRunner.HIVE_CATALOG; import static io.trino.plugin.hive.HiveQueryRunner.TPCH_SCHEMA; import static io.trino.plugin.hive.HiveQueryRunner.createBucketedSession; +import static io.trino.plugin.hive.HiveStorageFormat.ESRI; import static io.trino.plugin.hive.HiveStorageFormat.ORC; import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; import static io.trino.plugin.hive.HiveStorageFormat.REGEX; @@ -5701,6 +5702,7 @@ private boolean isMappingByName(HiveStorageFormat format) case TEXTFILE -> false; case CSV -> false; case REGEX -> false; + case ESRI -> true; }; } @@ -9505,6 +9507,10 @@ private List getAllTestingHiveStorageFormat() // REGEX format is read-only continue; } + if (hiveStorageFormat == ESRI) { + // ESRI format is read-only + continue; + } formats.add(new TestingHiveStorageFormat(getSession(), hiveStorageFormat)); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestEsriTable.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestEsriTable.java new file mode 100644 index 000000000000..a15e034bc9b2 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestEsriTable.java @@ -0,0 +1,120 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License 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 io.trino.plugin.hive; + +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Resources; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.MaterializedResult; +import io.trino.testing.MaterializedRow; +import io.trino.testing.QueryRunner; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.io.OutputStream; +import java.net.URL; +import java.util.Arrays; +import java.util.HexFormat; +import java.util.List; +import java.util.UUID; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; +import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; +import static java.nio.charset.StandardCharsets.UTF_8; + +public class TestEsriTable + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return HiveQueryRunner.builder() + .setHiveProperties(ImmutableMap.of("hive.non-managed-table-writes-enabled", "true")) + .build(); + } + + @Test + public void testCreateExternalTableWithData() + throws IOException + { + URL resourceLocation = Resources.getResource("esri/counties.json"); + TrinoFileSystem fileSystem = getConnectorService(getQueryRunner(), TrinoFileSystemFactory.class).create(ConnectorIdentity.ofUser("test")); + + // Create a temporary directory for the table data + Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); + fileSystem.createDirectory(tempDir); + Location dataFile = tempDir.appendPath("counties.json"); + + try (OutputStream out = fileSystem.newOutputFile(dataFile).create()) { + Resources.copy(resourceLocation, out); + } + + List expected = readExpectedResults("esri/counties_expected.txt"); + + // ESRI format is read-only, so create data files using the text file format + @Language("SQL") String createCountiesTableSql = + """ + CREATE TABLE counties ( + name varchar, boundaryshape varbinary) + WITH ( + format = 'esri', + external_location = '%s') + """.formatted(dataFile.parentDirectory()); + assertUpdate(createCountiesTableSql); + + MaterializedResult result = computeActual("SELECT * FROM counties"); + + assertEqualsIgnoreOrder(result.getMaterializedRows(), expected); + + assertQueryFails( + "INSERT INTO counties VALUES ('esri fails writes', X'0102030405')", + "Writing not supported for StorageFormat\\{serde=com\\.esri\\.hadoop\\.hive\\.serde\\.EsriJsonSerDe, inputFormat=com\\.esri\\.json\\.hadoop\\.EnclosedEsriJsonInputFormat, outputFormat=org\\.apache\\.hadoop\\.hive\\.ql\\.io\\.HiveIgnoreKeyTextOutputFormat\\}" + ); + + assertUpdate("DROP TABLE counties"); + } + + + private static List readExpectedResults(String resourcePath) + throws IOException + { + URL resourceUrl = Resources.getResource(resourcePath); + List lines = Resources.readLines(resourceUrl, UTF_8); + + return lines.stream() + .map(line -> { + String[] parts = line.split("\t"); // Assuming tab-separated values + return new MaterializedRow(Arrays.asList( + parts[0], // name + hexToBytes(parts[1]) // hex string for boundaryshape + )); + }) + .collect(toImmutableList()); + } + + private static byte[] hexToBytes(String hex) { + // Remove 'X' prefix, spaces, and single quotes if present + hex = hex.replaceAll("^X'|'$", "") // Remove X' and trailing ' + .replaceAll("\\s+", ""); // Remove all whitespace + + return HexFormat.of().parseHex(hex); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java index 13d421b0bee1..aa49e122ff37 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java @@ -122,6 +122,10 @@ void testAllFormats() // REGEX format is readonly continue; } + if (format == HiveStorageFormat.ESRI) { + // ESRI format is readonly + continue; + } config.setHiveStorageFormat(format); config.setHiveCompressionCodec(NONE); long uncompressedLength = writeTestFile(fileSystemFactory, config, sortingFileWriterConfig, metastore, makeFileName(config)); diff --git a/plugin/trino-hive/src/test/resources/esri/counties.json b/plugin/trino-hive/src/test/resources/esri/counties.json new file mode 100644 index 000000000000..a97cdbaec23a --- /dev/null +++ b/plugin/trino-hive/src/test/resources/esri/counties.json @@ -0,0 +1,5142 @@ +{ + "displayFieldName" : "", + "fieldAliases" : { + "OBJECTID" : "OBJECTID", + "AREA" : "AREA", + "PERIMETER" : "PERIMETER", + "CO06_D00_" : "CO06_D00_", + "CO06_D00_I" : "CO06_D00_I", + "STATE" : "STATE", + "COUNTY" : "COUNTY", + "NAME" : "NAME", + "LSAD" : "LSAD", + "LSAD_TRANS" : "LSAD_TRANS", + "Shape_Length" : "Shape_Length", + "Shape_Area" : "Shape_Area" + }, + "geometryType" : "esriGeometryPolygon", + "spatialReference" : { + "wkid" : null + }, + "fields" : [ + { + "name" : "OBJECTID", + "type" : "esriFieldTypeOID", + "alias" : "OBJECTID" + }, + { + "name" : "AREA", + "type" : "esriFieldTypeDouble", + "alias" : "AREA" + }, + { + "name" : "PERIMETER", + "type" : "esriFieldTypeDouble", + "alias" : "PERIMETER" + }, + { + "name" : "CO06_D00_", + "type" : "esriFieldTypeDouble", + "alias" : "CO06_D00_" + }, + { + "name" : "CO06_D00_I", + "type" : "esriFieldTypeDouble", + "alias" : "CO06_D00_I" + }, + { + "name" : "STATE", + "type" : "esriFieldTypeString", + "alias" : "STATE", + "length" : 2 + }, + { + "name" : "COUNTY", + "type" : "esriFieldTypeString", + "alias" : "COUNTY", + "length" : 3 + }, + { + "name" : "NAME", + "type" : "esriFieldTypeString", + "alias" : "NAME", + "length" : 90 + }, + { + "name" : "LSAD", + "type" : "esriFieldTypeString", + "alias" : "LSAD", + "length" : 2 + }, + { + "name" : "LSAD_TRANS", + "type" : "esriFieldTypeString", + "alias" : "LSAD_TRANS", + "length" : 50 + }, + { + "name" : "Shape_Length", + "type" : "esriFieldTypeDouble", + "alias" : "Shape_Length" + }, + { + "name" : "Shape_Area", + "type" : "esriFieldTypeDouble", + "alias" : "Shape_Area" + } + ], + "features" : [ + { + "attributes" : { + "datecol": 1741034025839, + "timestampcol": 1741034025839, + "OBJECTID" : 39, + "AREA" : 0.0125060450672465, + "PERIMETER" : 0.46770550097881602, + "CO06_D00_" : 40, + "CO06_D00_I" : 39, + "STATE" : "06", + "COUNTY" : "075", + "NAME" : "San Francisco", + "LSAD" : "06", + "LSAD_TRANS" : "County", + "Shape_Length" : 0.46770550097881591, + "Shape_Area" : 0.012506045067245427 + }, + "geometry" : { + "rings" : [ + [ + [ + -122.5024267151224, + 37.708132349276738 + ], + [ + -122.506483, + 37.723731000000001 + ], + [ + -122.50782901995821, + 37.735330999999753 + ], + [ + -122.50939539266651, + 37.74882999999992 + ], + [ + -122.50939990408358, + 37.748868879392219 + ], + [ + -122.51113374917077, + 37.763811156353654 + ], + [ + -122.511983, + 37.771129999999999 + ], + [ + -122.514483, + 37.780828999999997 + ], + [ + -122.50985492674391, + 37.7846044139513 + ], + [ + -122.50530999999999, + 37.788311999999998 + ], + [ + -122.4929995748537, + 37.787932592835674 + ], + [ + -122.49288300000001, + 37.787928999999998 + ], + [ + -122.485783, + 37.790629000000003 + ], + [ + -122.478083, + 37.810828000000001 + ], + [ + -122.47033619921733, + 37.80867139189769 + ], + [ + -122.46379252780997, + 37.804652871814127 + ], + [ + -122.44826217281084, + 37.807248691243323 + ], + [ + -122.44286017365187, + 37.808151607781561 + ], + [ + -122.4259424176653, + 37.810979324191834 + ], + [ + -122.42423175360105, + 37.81102202801582 + ], + [ + -122.42028987677702, + 37.81112043027818 + ], + [ + -122.40745179565363, + 37.811440911176838 + ], + [ + -122.3996578257948, + 37.806578201534457 + ], + [ + -122.39813860391477, + 37.805630348897253 + ], + [ + -122.39439124521003, + 37.801271662581449 + ], + [ + -122.38981779869818, + 37.795952124128398 + ], + [ + -122.38856208578255, + 37.794491559732741 + ], + [ + -122.38559109136668, + 37.791035890359986 + ], + [ + -122.3853227835059, + 37.790723811935322 + ], + [ + -122.38406388221165, + 37.783312374344398 + ], + [ + -122.38135678738803, + 37.767375092819996 + ], + [ + -122.37900260524107, + 37.753515488306547 + ], + [ + -122.37679663286187, + 37.740528428327821 + ], + [ + -122.3758876818182, + 37.73517722727334 + ], + [ + -122.37585395583966, + 37.7349786747845 + ], + [ + -122.37335481238509, + 37.733829999999998 + ], + [ + -122.37009367469328, + 37.732331091792666 + ], + [ + -122.37008910062909, + 37.732336075471807 + ], + [ + -122.3696210704221, + 37.732846018550809 + ], + [ + -122.36769653795419, + 37.73494289605506 + ], + [ + -122.36768710100542, + 37.734941526175874 + ], + [ + -122.36547829416895, + 37.734620893036777 + ], + [ + -122.3567842057419, + 37.729504618706976 + ], + [ + -122.36174893455313, + 37.715009522949558 + ], + [ + -122.37041125718375, + 37.717572339902105 + ], + [ + -122.37520618808058, + 37.715458494269519 + ], + [ + -122.39137380314342, + 37.708331000000001 + ], + [ + -122.39268199999999, + 37.708331000000001 + ], + [ + -122.393782, + 37.708230999999998 + ], + [ + -122.39518200000001, + 37.708331000000001 + ], + [ + -122.395782, + 37.708331000000001 + ], + [ + -122.405226, + 37.708271000000003 + ], + [ + -122.40545299999999, + 37.708244999999998 + ], + [ + -122.405582, + 37.708230999999998 + ], + [ + -122.41037, + 37.708283000000002 + ], + [ + -122.413282, + 37.708331000000001 + ], + [ + -122.41399699999999, + 37.708252000000002 + ], + [ + -122.414182, + 37.708230999999998 + ], + [ + -122.41448200000001, + 37.708230999999998 + ], + [ + -122.415182, + 37.708230999999998 + ], + [ + -122.416082, + 37.708331000000001 + ], + [ + -122.42008199999999, + 37.708230999999998 + ], + [ + -122.423782, + 37.708230999999998 + ], + [ + -122.42556500000001, + 37.708314000000001 + ], + [ + -122.425792, + 37.708323999999998 + ], + [ + -122.425882, + 37.708328000000002 + ], + [ + -122.426366, + 37.708351 + ], + [ + -122.426833, + 37.708373000000002 + ], + [ + -122.428082, + 37.708430999999997 + ], + [ + -122.430027, + 37.708280999999999 + ], + [ + -122.430882, + 37.708230999999998 + ], + [ + -122.43338199999999, + 37.708232000000002 + ], + [ + -122.435382, + 37.708131999999999 + ], + [ + -122.439144, + 37.708278 + ], + [ + -122.440082, + 37.708331999999999 + ], + [ + -122.440782, + 37.708331999999999 + ], + [ + -122.441282, + 37.708331999999999 + ], + [ + -122.442082, + 37.708232000000002 + ], + [ + -122.446983, + 37.708232000000002 + ], + [ + -122.44713400000001, + 37.708232000000002 + ], + [ + -122.44928299999999, + 37.708232000000002 + ], + [ + -122.45218300000001, + 37.708131999999999 + ], + [ + -122.45398299999999, + 37.708232000000002 + ], + [ + -122.45828299999999, + 37.708232000000002 + ], + [ + -122.461383, + 37.708232000000002 + ], + [ + -122.466283, + 37.708131999999999 + ], + [ + -122.46718300000001, + 37.708232000000002 + ], + [ + -122.467883, + 37.708232000000002 + ], + [ + -122.46888300000001, + 37.708232000000002 + ], + [ + -122.470883, + 37.708232000000002 + ], + [ + -122.47148300000001, + 37.708331999999999 + ], + [ + -122.481083, + 37.708232000000002 + ], + [ + -122.485383, + 37.708232000000002 + ], + [ + -122.48608299999999, + 37.708232000000002 + ], + [ + -122.5024267151224, + 37.708132349276738 + ] + ] + ] + } + }, + { + "attributes" : { + "OBJECTID" : 40, + "AREA" : 0.56617988740499303, + "PERIMETER" : 4.7681491757676397, + "CO06_D00_" : 41, + "CO06_D00_I" : 40, + "STATE" : "06", + "COUNTY" : "039", + "NAME" : "Madera", + "LSAD" : "06", + "LSAD_TRANS" : "County", + "Shape_Length" : 4.7681491757676353, + "Shape_Area" : 0.56617988740500069 + }, + "geometry" : { + "rings" : [ + [ + [ + -119.268979, + 37.739229999999999 + ], + [ + -119.264079, + 37.732931000000001 + ], + [ + -119.261179, + 37.732731000000001 + ], + [ + -119.26047800000001, + 37.731431000000001 + ], + [ + -119.260036, + 37.727173999999998 + ], + [ + -119.25787800000001, + 37.724631000000002 + ], + [ + -119.255178, + 37.725330999999997 + ], + [ + -119.25367799999999, + 37.728631 + ], + [ + -119.252978, + 37.729430999999998 + ], + [ + -119.250978, + 37.730159999999998 + ], + [ + -119.243045, + 37.728546999999999 + ], + [ + -119.23063, + 37.720953999999999 + ], + [ + -119.229992, + 37.717405999999997 + ], + [ + -119.22219, + 37.715051000000003 + ], + [ + -119.21633300000001, + 37.714123000000001 + ], + [ + -119.210928, + 37.721192000000002 + ], + [ + -119.206659, + 37.724882000000001 + ], + [ + -119.201717, + 37.731748000000003 + ], + [ + -119.20054, + 37.734915999999998 + ], + [ + -119.198491, + 37.736657000000001 + ], + [ + -119.19222499999999, + 37.737935999999998 + ], + [ + -119.18790799999999, + 37.736370000000001 + ], + [ + -119.182545, + 37.737093000000002 + ], + [ + -119.17634200000001, + 37.7348 + ], + [ + -119.17282400000001, + 37.735725000000002 + ], + [ + -119.171464, + 37.737811999999998 + ], + [ + -119.169071, + 37.738359000000003 + ], + [ + -119.166821, + 37.738050999999999 + ], + [ + -119.164075, + 37.736122999999999 + ], + [ + -119.155237, + 37.734641000000003 + ], + [ + -119.149593, + 37.732894999999999 + ], + [ + -119.14122999999999, + 37.733153999999999 + ], + [ + -119.127897, + 37.734681999999999 + ], + [ + -119.124312, + 37.733944000000001 + ], + [ + -119.114446, + 37.728194999999999 + ], + [ + -119.109694, + 37.724735000000003 + ], + [ + -119.107399, + 37.720072999999999 + ], + [ + -119.10455399999999, + 37.717725999999999 + ], + [ + -119.101973, + 37.715622000000003 + ], + [ + -119.101589, + 37.713647999999999 + ], + [ + -119.099979, + 37.71208 + ], + [ + -119.09685500000001, + 37.707647000000001 + ], + [ + -119.093126, + 37.703626999999997 + ], + [ + -119.09087100000001, + 37.702682000000003 + ], + [ + -119.073476, + 37.691225000000003 + ], + [ + -119.07130600000001, + 37.688792999999997 + ], + [ + -119.065315, + 37.683878 + ], + [ + -119.06277300000001, + 37.677424999999999 + ], + [ + -119.06244100000001, + 37.671095999999999 + ], + [ + -119.060619, + 37.669044 + ], + [ + -119.05963800000001, + 37.665047000000001 + ], + [ + -119.060311, + 37.663811000000003 + ], + [ + -119.059997, + 37.661344999999997 + ], + [ + -119.060671, + 37.659106000000001 + ], + [ + -119.06098900000001, + 37.658850999999999 + ], + [ + -119.057912, + 37.653879000000003 + ], + [ + -119.054331, + 37.651184000000001 + ], + [ + -119.0538, + 37.645781999999997 + ], + [ + -119.04963499999999, + 37.641902000000002 + ], + [ + -119.04348899999999, + 37.636425000000003 + ], + [ + -119.041813, + 37.634748999999999 + ], + [ + -119.03826100000001, + 37.632765999999997 + ], + [ + -119.036873, + 37.632534999999997 + ], + [ + -119.03299800000001, + 37.630617000000001 + ], + [ + -119.032341, + 37.629390000000001 + ], + [ + -119.032706, + 37.628653 + ], + [ + -119.031166, + 37.626373000000001 + ], + [ + -119.03088700000001, + 37.620891999999998 + ], + [ + -119.031966, + 37.613835999999999 + ], + [ + -119.033265, + 37.609735999999998 + ], + [ + -119.033342, + 37.609535000000001 + ], + [ + -119.033957, + 37.602882999999999 + ], + [ + -119.031864, + 37.600037 + ], + [ + -119.02656399999999, + 37.594737000000002 + ], + [ + -119.02616399999999, + 37.593437000000002 + ], + [ + -119.025164, + 37.590136999999999 + ], + [ + -119.022363, + 37.585737000000002 + ], + [ + -119.129985, + 37.500118000000001 + ], + [ + -119.130207, + 37.499941 + ], + [ + -119.18091800000001, + 37.458882000000003 + ], + [ + -119.28670099999999, + 37.374904999999998 + ], + [ + -119.300911, + 37.362721000000001 + ], + [ + -119.306181, + 37.356808999999998 + ], + [ + -119.31139, + 37.353541999999997 + ], + [ + -119.314491, + 37.349634000000002 + ], + [ + -119.31520999999999, + 37.348008999999998 + ], + [ + -119.315015, + 37.346037000000003 + ], + [ + -119.311539, + 37.342939000000001 + ], + [ + -119.311431, + 37.34075 + ], + [ + -119.312226, + 37.339706 + ], + [ + -119.316839, + 37.337783999999999 + ], + [ + -119.32441300000001, + 37.337060000000001 + ], + [ + -119.325774, + 37.335417 + ], + [ + -119.32580799999999, + 37.333962 + ], + [ + -119.324545, + 37.332509999999999 + ], + [ + -119.31644300000001, + 37.328127000000002 + ], + [ + -119.315575, + 37.325800999999998 + ], + [ + -119.316411, + 37.323331000000003 + ], + [ + -119.318789, + 37.320996000000001 + ], + [ + -119.32285899999999, + 37.321179000000001 + ], + [ + -119.32383299999999, + 37.320813000000001 + ], + [ + -119.331883, + 37.315182 + ], + [ + -119.33480400000001, + 37.312206000000003 + ], + [ + -119.33359799999999, + 37.304152000000002 + ], + [ + -119.33047500000001, + 37.302047000000002 + ], + [ + -119.326605, + 37.292710999999997 + ], + [ + -119.327607, + 37.288248000000003 + ], + [ + -119.329238, + 37.28445 + ], + [ + -119.33049800000001, + 37.282710999999999 + ], + [ + -119.331813, + 37.274884 + ], + [ + -119.331613, + 37.273876999999999 + ], + [ + -119.33104, + 37.273099000000002 + ], + [ + -119.326143, + 37.270330999999999 + ], + [ + -119.324252, + 37.265113999999997 + ], + [ + -119.324223, + 37.25882 + ], + [ + -119.32350700000001, + 37.256462999999997 + ], + [ + -119.32187399999999, + 37.253855000000001 + ], + [ + -119.322108, + 37.246388000000003 + ], + [ + -119.324054, + 37.244031 + ], + [ + -119.328461, + 37.242244999999997 + ], + [ + -119.33117900000001, + 37.239429999999999 + ], + [ + -119.332922, + 37.228307999999998 + ], + [ + -119.333636, + 37.226294000000003 + ], + [ + -119.333607, + 37.223914000000001 + ], + [ + -119.337182, + 37.220047000000001 + ], + [ + -119.337582, + 37.218628000000002 + ], + [ + -119.33452, + 37.216157000000003 + ], + [ + -119.330743, + 37.215128 + ], + [ + -119.329368, + 37.210025000000002 + ], + [ + -119.32999700000001, + 37.207073000000001 + ], + [ + -119.331226, + 37.205081999999997 + ], + [ + -119.337232, + 37.200023000000002 + ], + [ + -119.339404, + 37.193981000000001 + ], + [ + -119.343006, + 37.189357999999999 + ], + [ + -119.348326, + 37.186861999999998 + ], + [ + -119.353388, + 37.186127999999997 + ], + [ + -119.35942, + 37.181570999999998 + ], + [ + -119.360562, + 37.179167999999997 + ], + [ + -119.360355, + 37.169533999999999 + ], + [ + -119.362613, + 37.167060999999997 + ], + [ + -119.364756, + 37.165847999999997 + ], + [ + -119.367272, + 37.165114000000003 + ], + [ + -119.37342, + 37.164836999999999 + ], + [ + -119.374306, + 37.164448 + ], + [ + -119.37536299999999, + 37.163463 + ], + [ + -119.375626, + 37.16225 + ], + [ + -119.375497, + 37.159939000000001 + ], + [ + -119.374673, + 37.158268999999997 + ], + [ + -119.375015, + 37.157468000000001 + ], + [ + -119.379025, + 37.155343999999999 + ], + [ + -119.38379999999999, + 37.154065000000003 + ], + [ + -119.384744, + 37.153081 + ], + [ + -119.386004, + 37.149901 + ], + [ + -119.388749, + 37.14949 + ], + [ + -119.393066, + 37.149811999999997 + ], + [ + -119.399787, + 37.149835000000003 + ], + [ + -119.405097, + 37.158855000000003 + ], + [ + -119.407983, + 37.160910999999999 + ], + [ + -119.418183, + 37.164062000000001 + ], + [ + -119.425805, + 37.164296999999998 + ], + [ + -119.430697, + 37.163142000000001 + ], + [ + -119.432125, + 37.162571 + ], + [ + -119.43306, + 37.161515999999999 + ], + [ + -119.43310099999999, + 37.152016000000003 + ], + [ + -119.433939, + 37.148291 + ], + [ + -119.43487500000001, + 37.146954000000001 + ], + [ + -119.44271999999999, + 37.144050999999997 + ], + [ + -119.450469, + 37.144945999999997 + ], + [ + -119.456665, + 37.144996999999996 + ], + [ + -119.460188, + 37.145584999999997 + ], + [ + -119.46150299999999, + 37.145493000000002 + ], + [ + -119.462189, + 37.144897999999998 + ], + [ + -119.463987, + 37.138216999999997 + ], + [ + -119.46812799999999, + 37.129406000000003 + ], + [ + -119.46887, + 37.126475999999997 + ], + [ + -119.46878700000001, + 37.124948000000003 + ], + [ + -119.46613000000001, + 37.120835999999997 + ], + [ + -119.46618599999999, + 37.119256999999998 + ], + [ + -119.46975399999999, + 37.110903999999998 + ], + [ + -119.471154, + 37.110194 + ], + [ + -119.481067, + 37.109321000000001 + ], + [ + -119.482097, + 37.109572 + ], + [ + -119.484326, + 37.111241 + ], + [ + -119.485843, + 37.114283999999998 + ], + [ + -119.488731, + 37.117921000000003 + ], + [ + -119.49119, + 37.119728000000002 + ], + [ + -119.49302299999999, + 37.124349000000002 + ], + [ + -119.491421, + 37.131067999999999 + ], + [ + -119.489622, + 37.133310999999999 + ], + [ + -119.48882399999999, + 37.135486 + ], + [ + -119.48913899999999, + 37.137338999999997 + ], + [ + -119.49006300000001, + 37.138579999999997 + ], + [ + -119.496404, + 37.144269000000001 + ], + [ + -119.499264, + 37.144725000000001 + ], + [ + -119.50330200000001, + 37.147328999999999 + ], + [ + -119.503783, + 37.147978999999999 + ], + [ + -119.50435400000001, + 37.149064000000003 + ], + [ + -119.50604800000001, + 37.150345000000002 + ], + [ + -119.50932899999999, + 37.150500999999998 + ], + [ + -119.517838, + 37.144559000000001 + ], + [ + -119.523551, + 37.129286999999998 + ], + [ + -119.525006, + 37.128250000000001 + ], + [ + -119.527575, + 37.129019 + ], + [ + -119.532832, + 37.133046999999998 + ], + [ + -119.539919, + 37.135246000000002 + ], + [ + -119.540976, + 37.136412999999997 + ], + [ + -119.541918, + 37.138587000000001 + ], + [ + -119.54612, + 37.142592 + ], + [ + -119.550893, + 37.145178000000001 + ], + [ + -119.552665, + 37.145384 + ], + [ + -119.55958200000001, + 37.143828999999997 + ], + [ + -119.562212, + 37.141585999999997 + ], + [ + -119.563613, + 37.138840000000002 + ], + [ + -119.563213, + 37.134996000000001 + ], + [ + -119.56355600000001, + 37.128405999999998 + ], + [ + -119.56616200000001, + 37.122323999999999 + ], + [ + -119.568904, + 37.1188 + ], + [ + -119.568904, + 37.117542 + ], + [ + -119.568361, + 37.116672000000001 + ], + [ + -119.566847, + 37.115963000000001 + ], + [ + -119.555674, + 37.115597000000001 + ], + [ + -119.551474, + 37.117130000000003 + ], + [ + -119.549273, + 37.117016 + ], + [ + -119.54755900000001, + 37.116123000000002 + ], + [ + -119.53995999999999, + 37.109211999999999 + ], + [ + -119.53770400000001, + 37.105069999999998 + ], + [ + -119.539277, + 37.099716000000001 + ], + [ + -119.541391, + 37.096215000000001 + ], + [ + -119.543105, + 37.095323 + ], + [ + -119.54510500000001, + 37.095345999999999 + ], + [ + -119.549447, + 37.094110999999998 + ], + [ + -119.554675, + 37.091822999999998 + ], + [ + -119.559045, + 37.088183999999998 + ], + [ + -119.561272, + 37.079146999999999 + ], + [ + -119.563357, + 37.074821999999998 + ], + [ + -119.56310000000001, + 37.073540999999999 + ], + [ + -119.561244, + 37.071939 + ], + [ + -119.56057800000001, + 37.070728000000003 + ], + [ + -119.56161, + 37.065323999999997 + ], + [ + -119.562979, + 37.064095000000002 + ], + [ + -119.568416, + 37.063569000000001 + ], + [ + -119.580749, + 37.066102999999998 + ], + [ + -119.584847, + 37.065621999999998 + ], + [ + -119.589431, + 37.066636000000003 + ], + [ + -119.59454700000001, + 37.070186999999997 + ], + [ + -119.59696099999999, + 37.071266000000001 + ], + [ + -119.600053, + 37.071634000000003 + ], + [ + -119.60490799999999, + 37.071016 + ], + [ + -119.606077, + 37.070374999999999 + ], + [ + -119.60966999999999, + 37.064692000000001 + ], + [ + -119.610624, + 37.062024999999998 + ], + [ + -119.61032, + 37.059643999999999 + ], + [ + -119.61063900000001, + 37.057768000000003 + ], + [ + -119.61319399999999, + 37.053502000000002 + ], + [ + -119.613602, + 37.051974999999999 + ], + [ + -119.613373, + 37.050778000000001 + ], + [ + -119.610361, + 37.045667999999999 + ], + [ + -119.610738, + 37.044651999999999 + ], + [ + -119.61921700000001, + 37.028955000000003 + ], + [ + -119.62065699999999, + 37.027057999999997 + ], + [ + -119.62243700000001, + 37.026074000000001 + ], + [ + -119.625974, + 37.025181000000003 + ], + [ + -119.632172, + 37.020983000000001 + ], + [ + -119.634064, + 37.020890999999999 + ], + [ + -119.63537100000001, + 37.021545000000003 + ], + [ + -119.63618, + 37.023696000000001 + ], + [ + -119.635953, + 37.025345000000002 + ], + [ + -119.63035600000001, + 37.031841999999997 + ], + [ + -119.62903300000001, + 37.034618999999999 + ], + [ + -119.639349, + 37.043286999999999 + ], + [ + -119.64748899999999, + 37.043774999999997 + ], + [ + -119.649456, + 37.043497000000002 + ], + [ + -119.65160400000001, + 37.042960999999998 + ], + [ + -119.653999, + 37.041176999999998 + ], + [ + -119.65937700000001, + 37.038938000000002 + ], + [ + -119.65711400000001, + 37.018903000000002 + ], + [ + -119.657359, + 37.016399999999997 + ], + [ + -119.65861099999999, + 37.013330000000003 + ], + [ + -119.662195, + 37.011758999999998 + ], + [ + -119.670918, + 37.009259999999998 + ], + [ + -119.677362, + 37.009107999999998 + ], + [ + -119.684794, + 37.010382999999997 + ], + [ + -119.69077900000001, + 37.011986999999998 + ], + [ + -119.693867, + 37.010955000000003 + ], + [ + -119.698075, + 37.008747999999997 + ], + [ + -119.69929999999999, + 37.007275999999997 + ], + [ + -119.69971, + 37.004904000000003 + ], + [ + -119.700744, + 37.003328000000003 + ], + [ + -119.705878, + 36.999949000000001 + ], + [ + -119.713078, + 36.991649000000002 + ], + [ + -119.73144600000001, + 36.979238000000002 + ], + [ + -119.73277899999999, + 36.978349000000001 + ], + [ + -119.74047899999999, + 36.969949 + ], + [ + -119.742679, + 36.953749000000002 + ], + [ + -119.738196, + 36.953870999999999 + ], + [ + -119.736609, + 36.953153999999998 + ], + [ + -119.733778, + 36.948549 + ], + [ + -119.734273, + 36.946753000000001 + ], + [ + -119.735433, + 36.945303000000003 + ], + [ + -119.737679, + 36.943949000000003 + ], + [ + -119.739879, + 36.944149000000003 + ], + [ + -119.744771, + 36.940328999999998 + ], + [ + -119.747179, + 36.938448999999999 + ], + [ + -119.75197900000001, + 36.935848999999997 + ], + [ + -119.751879, + 36.932149000000003 + ], + [ + -119.750479, + 36.928449000000001 + ], + [ + -119.752779, + 36.923648999999997 + ], + [ + -119.756979, + 36.920549000000001 + ], + [ + -119.761779, + 36.920048999999999 + ], + [ + -119.76608, + 36.918849000000002 + ], + [ + -119.77258, + 36.918548999999999 + ], + [ + -119.77408, + 36.914749 + ], + [ + -119.77528, + 36.909249000000003 + ], + [ + -119.78818, + 36.897649000000001 + ], + [ + -119.78958, + 36.894249000000002 + ], + [ + -119.78878, + 36.892549000000002 + ], + [ + -119.78848000000001, + 36.890448999999997 + ], + [ + -119.78558, + 36.885649000000001 + ], + [ + -119.78548000000001, + 36.879949000000003 + ], + [ + -119.78908, + 36.875948999999999 + ], + [ + -119.79147, + 36.875948999999999 + ], + [ + -119.79178, + 36.875948999999999 + ], + [ + -119.791848, + 36.875940999999997 + ], + [ + -119.793479, + 36.875745999999999 + ], + [ + -119.79844799999999, + 36.872537000000001 + ], + [ + -119.80148, + 36.868448000000001 + ], + [ + -119.805026, + 36.864348 + ], + [ + -119.805181, + 36.864148 + ], + [ + -119.809281, + 36.862748000000003 + ], + [ + -119.812181, + 36.857948 + ], + [ + -119.812681, + 36.852448000000003 + ], + [ + -119.81309299999999, + 36.852114 + ], + [ + -119.818781, + 36.848148000000002 + ], + [ + -119.82328099999999, + 36.848647999999997 + ], + [ + -119.831895, + 36.851505000000003 + ], + [ + -119.835982, + 36.853248000000001 + ], + [ + -119.837682, + 36.854948 + ], + [ + -119.839282, + 36.858747999999999 + ], + [ + -119.840782, + 36.860948 + ], + [ + -119.84298200000001, + 36.861248000000003 + ], + [ + -119.84463700000001, + 36.860804000000002 + ], + [ + -119.847083, + 36.860148000000002 + ], + [ + -119.848383, + 36.858747999999999 + ], + [ + -119.849283, + 36.856347999999997 + ], + [ + -119.85298299999999, + 36.851748000000001 + ], + [ + -119.854783, + 36.851548000000001 + ], + [ + -119.856683, + 36.852547999999999 + ], + [ + -119.856683, + 36.852046999999999 + ], + [ + -119.85763799999999, + 36.852797000000002 + ], + [ + -119.861383, + 36.855246999999999 + ], + [ + -119.86528300000001, + 36.851146999999997 + ], + [ + -119.865883, + 36.847946999999998 + ], + [ + -119.866401, + 36.84798 + ], + [ + -119.868984, + 36.848146999999997 + ], + [ + -119.86888399999999, + 36.850346999999999 + ], + [ + -119.876484, + 36.854846999999999 + ], + [ + -119.884885, + 36.858547000000002 + ], + [ + -119.885908, + 36.857954999999997 + ], + [ + -119.887885, + 36.854947000000003 + ], + [ + -119.887944, + 36.853645 + ], + [ + -119.889285, + 36.851947000000003 + ], + [ + -119.895685, + 36.852147000000002 + ], + [ + -119.896085, + 36.852646999999997 + ], + [ + -119.902185, + 36.852046999999999 + ], + [ + -119.905986, + 36.850346999999999 + ], + [ + -119.909486, + 36.846347000000002 + ], + [ + -119.91388600000001, + 36.845146999999997 + ], + [ + -119.918486, + 36.845446000000003 + ], + [ + -119.923286, + 36.847946 + ], + [ + -119.92728700000001, + 36.848646000000002 + ], + [ + -119.930587, + 36.846746000000003 + ], + [ + -119.932587, + 36.844346000000002 + ], + [ + -119.932087, + 36.843546000000003 + ], + [ + -119.932587, + 36.842945999999998 + ], + [ + -119.934687, + 36.837946000000002 + ], + [ + -119.936987, + 36.836246000000003 + ], + [ + -119.943487, + 36.834145999999997 + ], + [ + -119.952377, + 36.834468000000001 + ], + [ + -119.957977, + 36.836222999999997 + ], + [ + -119.96109, + 36.835299999999997 + ], + [ + -119.967285, + 36.832393000000003 + ], + [ + -119.968765, + 36.832301000000001 + ], + [ + -119.970529, + 36.832884999999997 + ], + [ + -119.976884, + 36.835031999999998 + ], + [ + -119.97963, + 36.839739000000002 + ], + [ + -119.98451300000001, + 36.841028000000001 + ], + [ + -119.99022100000001, + 36.837969000000001 + ], + [ + -119.991518, + 36.835557999999999 + ], + [ + -119.99139599999999, + 36.832307 + ], + [ + -119.990647, + 36.831117999999996 + ], + [ + -119.99128899999999, + 36.829897000000003 + ], + [ + -119.99249500000001, + 36.829065 + ], + [ + -119.99910199999999, + 36.827401999999999 + ], + [ + -120.006603, + 36.827407999999998 + ], + [ + -120.00842400000001, + 36.828004 + ], + [ + -120.01329, + 36.828234999999999 + ], + [ + -120.02029400000001, + 36.824738000000004 + ], + [ + -120.02112, + 36.823228 + ], + [ + -120.021976, + 36.819065000000002 + ], + [ + -120.024649, + 36.816567999999997 + ], + [ + -120.025364, + 36.815565999999997 + ], + [ + -120.027356, + 36.814720000000001 + ], + [ + -120.040843, + 36.815227 + ], + [ + -120.045878, + 36.817515999999998 + ], + [ + -120.05077199999999, + 36.821863999999998 + ], + [ + -120.057794, + 36.822932999999999 + ], + [ + -120.06269500000001, + 36.820538999999997 + ], + [ + -120.070179, + 36.818457000000002 + ], + [ + -120.073424, + 36.820126999999999 + ], + [ + -120.076099, + 36.824542000000001 + ], + [ + -120.07982699999999, + 36.825319999999998 + ], + [ + -120.09066900000001, + 36.821748999999997 + ], + [ + -120.095107, + 36.817790000000002 + ], + [ + -120.09635, + 36.817227000000003 + ], + [ + -120.09798000000001, + 36.815364000000002 + ], + [ + -120.10808, + 36.812775999999999 + ], + [ + -120.111011, + 36.813049999999997 + ], + [ + -120.11449, + 36.814090999999998 + ], + [ + -120.12911200000001, + 36.811805 + ], + [ + -120.136341, + 36.809544000000002 + ], + [ + -120.14402699999999, + 36.804651999999997 + ], + [ + -120.147299, + 36.803806999999999 + ], + [ + -120.152052, + 36.801223 + ], + [ + -120.156178, + 36.798662 + ], + [ + -120.156947, + 36.797609000000001 + ], + [ + -120.15893800000001, + 36.79777 + ], + [ + -120.162863, + 36.799075000000002 + ], + [ + -120.163659, + 36.800311000000001 + ], + [ + -120.168618, + 36.803804999999997 + ], + [ + -120.170743, + 36.803812999999998 + ], + [ + -120.172707, + 36.80283 + ], + [ + -120.173703, + 36.800655999999996 + ], + [ + -120.173192, + 36.796629000000003 + ], + [ + -120.173591, + 36.795394000000002 + ], + [ + -120.17447300000001, + 36.794296000000003 + ], + [ + -120.177773, + 36.792876999999997 + ], + [ + -120.181443, + 36.789766 + ], + [ + -120.181955, + 36.786997999999997 + ], + [ + -120.180988, + 36.786403 + ], + [ + -120.181358, + 36.784503999999998 + ], + [ + -120.18272399999999, + 36.782283999999997 + ], + [ + -120.188953, + 36.776862000000001 + ], + [ + -120.19228099999999, + 36.777205000000002 + ], + [ + -120.199477, + 36.780751000000002 + ], + [ + -120.200785, + 36.782100999999997 + ], + [ + -120.20237899999999, + 36.786310999999998 + ], + [ + -120.20178300000001, + 36.789605999999999 + ], + [ + -120.20220999999999, + 36.790018000000003 + ], + [ + -120.205339, + 36.790954999999997 + ], + [ + -120.210942, + 36.788805000000004 + ], + [ + -120.213418, + 36.790131000000002 + ], + [ + -120.216291, + 36.790610999999998 + ], + [ + -120.220074, + 36.788573999999997 + ], + [ + -120.220271, + 36.785370999999998 + ], + [ + -120.224253, + 36.782853000000003 + ], + [ + -120.22667, + 36.782234000000003 + ], + [ + -120.230965, + 36.782508 + ], + [ + -120.23272799999999, + 36.781317000000001 + ], + [ + -120.23341000000001, + 36.779874999999997 + ], + [ + -120.233324, + 36.778686 + ], + [ + -120.231673, + 36.775436999999997 + ], + [ + -120.229963, + 36.770083999999997 + ], + [ + -120.23007699999999, + 36.769419999999997 + ], + [ + -120.231669, + 36.768825 + ], + [ + -120.234058, + 36.768571999999999 + ], + [ + -120.236846, + 36.770493000000002 + ], + [ + -120.238866, + 36.771064000000003 + ], + [ + -120.243672, + 36.770536 + ], + [ + -120.24506599999999, + 36.770786999999999 + ], + [ + -120.24777, + 36.773027999999996 + ], + [ + -120.24808400000001, + 36.774835000000003 + ], + [ + -120.250246, + 36.775086000000002 + ], + [ + -120.252037, + 36.773983999999999 + ], + [ + -120.254031, + 36.770896 + ], + [ + -120.255994, + 36.770096000000002 + ], + [ + -120.25804100000001, + 36.770349000000003 + ], + [ + -120.261196, + 36.772478 + ], + [ + -120.263273, + 36.772708000000002 + ], + [ + -120.27046900000001, + 36.771954999999998 + ], + [ + -120.27618699999999, + 36.770631000000002 + ], + [ + -120.279971, + 36.766491000000002 + ], + [ + -120.28093800000001, + 36.766240000000003 + ], + [ + -120.282161, + 36.766674999999999 + ], + [ + -120.2829, + 36.768025000000002 + ], + [ + -120.28267200000001, + 36.769855 + ], + [ + -120.28173200000001, + 36.771388000000002 + ], + [ + -120.281874, + 36.772281 + ], + [ + -120.284035, + 36.773584999999997 + ], + [ + -120.2854, + 36.773516999999998 + ], + [ + -120.288195, + 36.772331000000001 + ], + [ + -120.289427, + 36.770477999999997 + ], + [ + -120.289321, + 36.767454999999998 + ], + [ + -120.28795100000001, + 36.764823 + ], + [ + -120.28828799999999, + 36.763672999999997 + ], + [ + -120.289812, + 36.763058000000001 + ], + [ + -120.29182299999999, + 36.763449000000001 + ], + [ + -120.293623, + 36.765369999999997 + ], + [ + -120.294166, + 36.766815999999999 + ], + [ + -120.293627, + 36.769106999999998 + ], + [ + -120.292119, + 36.770816000000003 + ], + [ + -120.29215000000001, + 36.771388000000002 + ], + [ + -120.293171, + 36.772233 + ], + [ + -120.297695, + 36.772289000000001 + ], + [ + -120.302688, + 36.769745999999998 + ], + [ + -120.304078, + 36.769638 + ], + [ + -120.30605799999999, + 36.770265000000002 + ], + [ + -120.30638999999999, + 36.771647999999999 + ], + [ + -120.305308, + 36.773285000000001 + ], + [ + -120.300895, + 36.773727999999998 + ], + [ + -120.29980500000001, + 36.774571999999999 + ], + [ + -120.299667, + 36.776130000000002 + ], + [ + -120.30035700000001, + 36.777633000000002 + ], + [ + -120.312583, + 36.782024 + ], + [ + -120.31374599999999, + 36.781934 + ], + [ + -120.314538, + 36.779170000000001 + ], + [ + -120.31589200000001, + 36.778284999999997 + ], + [ + -120.317662, + 36.777695000000001 + ], + [ + -120.32172199999999, + 36.777697000000003 + ], + [ + -120.32382800000001, + 36.779018999999998 + ], + [ + -120.324349, + 36.779893000000001 + ], + [ + -120.324552, + 36.781599999999997 + ], + [ + -120.322264, + 36.784590999999999 + ], + [ + -120.322824, + 36.785573999999997 + ], + [ + -120.32421600000001, + 36.786304000000001 + ], + [ + -120.329252, + 36.785184000000001 + ], + [ + -120.33284999999999, + 36.782944999999998 + ], + [ + -120.334614, + 36.782884000000003 + ], + [ + -120.33556299999999, + 36.784170000000003 + ], + [ + -120.335312, + 36.786237 + ], + [ + -120.334264, + 36.788946000000003 + ], + [ + -120.33438700000001, + 36.791469999999997 + ], + [ + -120.33513000000001, + 36.792068999999998 + ], + [ + -120.33694, + 36.792395999999997 + ], + [ + -120.341832, + 36.791004999999998 + ], + [ + -120.344071, + 36.791384999999998 + ], + [ + -120.34607099999999, + 36.794733999999998 + ], + [ + -120.34635900000001, + 36.797285000000002 + ], + [ + -120.34766399999999, + 36.798161999999998 + ], + [ + -120.34983, + 36.798095000000004 + ], + [ + -120.35067100000001, + 36.797494999999998 + ], + [ + -120.35095, + 36.795436000000002 + ], + [ + -120.348206, + 36.789909000000002 + ], + [ + -120.344506, + 36.787413999999998 + ], + [ + -120.344335, + 36.786352000000001 + ], + [ + -120.345124, + 36.785049000000001 + ], + [ + -120.346523, + 36.784559000000002 + ], + [ + -120.349422, + 36.785134999999997 + ], + [ + -120.353095, + 36.786859 + ], + [ + -120.35588300000001, + 36.787348000000001 + ], + [ + -120.369052, + 36.784804999999999 + ], + [ + -120.37296000000001, + 36.789020999999998 + ], + [ + -120.370773, + 36.793140999999999 + ], + [ + -120.36943599999999, + 36.794423000000002 + ], + [ + -120.370377, + 36.796528000000002 + ], + [ + -120.373907, + 36.800141000000004 + ], + [ + -120.373993, + 36.801079999999999 + ], + [ + -120.372857, + 36.803573999999998 + ], + [ + -120.36822100000001, + 36.806986000000002 + ], + [ + -120.369163, + 36.809936999999998 + ], + [ + -120.371725, + 36.810417000000001 + ], + [ + -120.374939, + 36.808515999999997 + ], + [ + -120.376924, + 36.808672000000001 + ], + [ + -120.37809, + 36.809930999999999 + ], + [ + -120.378142, + 36.815193000000001 + ], + [ + -120.379193, + 36.817208000000001 + ], + [ + -120.388666, + 36.824373999999999 + ], + [ + -120.390772, + 36.824513000000003 + ], + [ + -120.397975, + 36.822364999999998 + ], + [ + -120.40028, + 36.822457 + ], + [ + -120.40139000000001, + 36.823121 + ], + [ + -120.404149, + 36.826005000000002 + ], + [ + -120.40411899999999, + 36.828637000000001 + ], + [ + -120.404886, + 36.831383000000002 + ], + [ + -120.409893, + 36.84008 + ], + [ + -120.411858, + 36.840057999999999 + ], + [ + -120.413425, + 36.8369 + ], + [ + -120.414991, + 36.836008 + ], + [ + -120.416613, + 36.83578 + ], + [ + -120.418463, + 36.836581000000002 + ], + [ + -120.419601, + 36.838617999999997 + ], + [ + -120.42088200000001, + 36.839692999999997 + ], + [ + -120.422247, + 36.840052 + ], + [ + -120.42526599999999, + 36.839992000000002 + ], + [ + -120.426376, + 36.840494999999997 + ], + [ + -120.42828299999999, + 36.842097000000003 + ], + [ + -120.42873899999999, + 36.843608000000003 + ], + [ + -120.433408, + 36.849215000000001 + ], + [ + -120.43944399999999, + 36.852144000000003 + ], + [ + -120.438219, + 36.854683999999999 + ], + [ + -120.441209, + 36.85624 + ], + [ + -120.44721699999999, + 36.856675000000003 + ], + [ + -120.448744, + 36.858328 + ], + [ + -120.44821399999999, + 36.859375 + ], + [ + -120.448499, + 36.860152999999997 + ], + [ + -120.453739, + 36.860359000000003 + ], + [ + -120.456445, + 36.862966999999998 + ], + [ + -120.450693, + 36.867773 + ], + [ + -120.449895, + 36.869238000000003 + ], + [ + -120.45003800000001, + 36.870725 + ], + [ + -120.452145, + 36.871983999999998 + ], + [ + -120.457671, + 36.873309999999996 + ], + [ + -120.459124, + 36.874645000000001 + ], + [ + -120.458905, + 36.876553999999999 + ], + [ + -120.457706, + 36.878219000000001 + ], + [ + -120.454089, + 36.879317999999998 + ], + [ + -120.453605, + 36.880760000000002 + ], + [ + -120.456227, + 36.883733999999997 + ], + [ + -120.464404, + 36.886822000000002 + ], + [ + -120.46654100000001, + 36.889659000000002 + ], + [ + -120.466571, + 36.892817000000001 + ], + [ + -120.462613, + 36.900345999999999 + ], + [ + -120.463697, + 36.905310999999998 + ], + [ + -120.463213, + 36.905746000000001 + ], + [ + -120.459851, + 36.906180999999997 + ], + [ + -120.455321, + 36.905678000000002 + ], + [ + -120.450563, + 36.910988000000003 + ], + [ + -120.450164, + 36.912384000000003 + ], + [ + -120.452045, + 36.914306000000003 + ], + [ + -120.45939799999999, + 36.915425999999997 + ], + [ + -120.46222, + 36.917347999999997 + ], + [ + -120.46239199999999, + 36.918469000000002 + ], + [ + -120.46193599999999, + 36.919384000000001 + ], + [ + -120.457633, + 36.921627999999998 + ], + [ + -120.456864, + 36.923161 + ], + [ + -120.45743400000001, + 36.924191 + ], + [ + -120.462034, + 36.927630999999998 + ], + [ + -120.465816, + 36.933456999999997 + ], + [ + -120.467185, + 36.936843000000003 + ], + [ + -120.469211, + 36.938032999999997 + ], + [ + -120.473743, + 36.938626999999997 + ], + [ + -120.47562499999999, + 36.940251000000004 + ], + [ + -120.478793, + 36.948259 + ], + [ + -120.47888, + 36.950273000000003 + ], + [ + -120.477769, + 36.953567999999997 + ], + [ + -120.478055, + 36.954780999999997 + ], + [ + -120.481594, + 36.960087999999999 + ], + [ + -120.484048, + 36.962696000000001 + ], + [ + -120.487984, + 36.965210999999996 + ], + [ + -120.48886899999999, + 36.966400999999998 + ], + [ + -120.488727, + 36.967475999999998 + ], + [ + -120.488128, + 36.968049000000001 + ], + [ + -120.48505, + 36.969766 + ], + [ + -120.48402400000001, + 36.971162 + ], + [ + -120.484081, + 36.972467000000002 + ], + [ + -120.493866, + 36.975827000000002 + ], + [ + -120.49475099999999, + 36.976512999999997 + ], + [ + -120.499404, + 36.981887999999998 + ], + [ + -120.500089, + 36.983330000000002 + ], + [ + -120.500007, + 36.988021000000003 + ], + [ + -120.501012, + 36.991199999999999 + ], + [ + -120.501548, + 36.993189000000001 + ], + [ + -120.500704, + 36.994405 + ], + [ + -120.500576, + 36.996029999999998 + ], + [ + -120.501486, + 36.996918999999998 + ], + [ + -120.508881, + 36.999946000000001 + ], + [ + -120.50967900000001, + 37.001052999999999 + ], + [ + -120.509764, + 37.00206 + ], + [ + -120.50853600000001, + 37.004142000000002 + ], + [ + -120.50853499999999, + 37.005766000000001 + ], + [ + -120.511613, + 37.009611999999997 + ], + [ + -120.524081, + 37.013553000000002 + ], + [ + -120.530787, + 37.012686000000002 + ], + [ + -120.531386, + 37.013761000000002 + ], + [ + -120.5299, + 37.017788000000003 + ], + [ + -120.53089799999999, + 37.019641999999997 + ], + [ + -120.53218200000001, + 37.020328999999997 + ], + [ + -120.53632, + 37.018911000000003 + ], + [ + -120.537633, + 37.019300999999999 + ], + [ + -120.53751800000001, + 37.023190999999997 + ], + [ + -120.544995, + 37.026418999999997 + ], + [ + -120.545365, + 37.027929 + ], + [ + -120.543395, + 37.030445999999998 + ], + [ + -120.54014100000001, + 37.032184000000001 + ], + [ + -120.538371, + 37.034151000000001 + ], + [ + -120.537684, + 37.039895000000001 + ], + [ + -120.542222, + 37.044083999999998 + ], + [ + -120.49238699999999, + 37.083955000000003 + ], + [ + -120.476692, + 37.09639 + ], + [ + -120.47246699999999, + 37.094141999999998 + ], + [ + -120.469779, + 37.094335999999998 + ], + [ + -120.465508, + 37.097777000000001 + ], + [ + -120.461366, + 37.099066000000001 + ], + [ + -120.456084, + 37.100343000000002 + ], + [ + -120.450777, + 37.100665999999997 + ], + [ + -120.441858, + 37.103746999999998 + ], + [ + -120.427787, + 37.106251999999998 + ], + [ + -120.421358, + 37.109664000000002 + ], + [ + -120.414816, + 37.110902000000003 + ], + [ + -120.40987199999999, + 37.111258999999997 + ], + [ + -120.40648, + 37.112290000000002 + ], + [ + -120.40409, + 37.115715000000002 + ], + [ + -120.400768, + 37.117759999999997 + ], + [ + -120.388831, + 37.120981999999998 + ], + [ + -120.385717, + 37.122864999999997 + ], + [ + -120.38593400000001, + 37.12377 + ], + [ + -120.38489, + 37.124589 + ], + [ + -120.37903300000001, + 37.124352000000002 + ], + [ + -120.372851, + 37.121496 + ], + [ + -120.365647, + 37.119965000000001 + ], + [ + -120.364131, + 37.120032999999999 + ], + [ + -120.35411499999999, + 37.123353999999999 + ], + [ + -120.341205, + 37.124597000000001 + ], + [ + -120.33577200000001, + 37.127214000000002 + ], + [ + -120.333563, + 37.128328000000003 + ], + [ + -120.328729, + 37.131967000000003 + ], + [ + -120.328609, + 37.133884999999999 + ], + [ + -120.327405, + 37.134872999999999 + ], + [ + -120.324546, + 37.136381 + ], + [ + -120.31475399999999, + 37.138337999999997 + ], + [ + -120.310892, + 37.141137999999998 + ], + [ + -120.30386300000001, + 37.141081 + ], + [ + -120.301821, + 37.143822999999998 + ], + [ + -120.301124, + 37.144444 + ], + [ + -120.298261, + 37.147522000000002 + ], + [ + -120.29299899999999, + 37.151079000000003 + ], + [ + -120.291957, + 37.151729000000003 + ], + [ + -120.290288, + 37.152453999999999 + ], + [ + -120.28141599999999, + 37.153013000000001 + ], + [ + -120.276039, + 37.152171000000003 + ], + [ + -120.27037900000001, + 37.153840000000002 + ], + [ + -120.265147, + 37.154454999999999 + ], + [ + -120.26494700000001, + 37.154134999999997 + ], + [ + -120.262259, + 37.155231999999998 + ], + [ + -120.24939500000001, + 37.157770999999997 + ], + [ + -120.242222, + 37.157961 + ], + [ + -120.235929, + 37.158703000000003 + ], + [ + -120.232277, + 37.159922000000002 + ], + [ + -120.23088, + 37.160848999999999 + ], + [ + -120.23040899999999, + 37.161782000000002 + ], + [ + -120.22728600000001, + 37.163400000000003 + ], + [ + -120.222965, + 37.163969000000002 + ], + [ + -120.21950099999999, + 37.162154000000001 + ], + [ + -120.213657, + 37.164318000000002 + ], + [ + -120.208707, + 37.164448 + ], + [ + -120.202135, + 37.162683999999999 + ], + [ + -120.19813600000001, + 37.163254000000002 + ], + [ + -120.19479699999999, + 37.163150999999999 + ], + [ + -120.187594, + 37.160252 + ], + [ + -120.18548699999999, + 37.161746999999998 + ], + [ + -120.180762, + 37.161512000000002 + ], + [ + -120.176489, + 37.162264999999998 + ], + [ + -120.175636, + 37.165059999999997 + ], + [ + -120.17412899999999, + 37.165928999999998 + ], + [ + -120.170226, + 37.165982 + ], + [ + -120.168544, + 37.164186999999998 + ], + [ + -120.166483, + 37.163066999999998 + ], + [ + -120.164576, + 37.162697999999999 + ], + [ + -120.157053, + 37.165525000000002 + ], + [ + -120.15295399999999, + 37.166021000000001 + ], + [ + -120.14654899999999, + 37.163851999999999 + ], + [ + -120.142563, + 37.164065000000001 + ], + [ + -120.13054, + 37.167155999999999 + ], + [ + -120.126003, + 37.166114 + ], + [ + -120.120099, + 37.165531999999999 + ], + [ + -120.115089, + 37.165655999999998 + ], + [ + -120.114538, + 37.16666 + ], + [ + -120.10986200000001, + 37.167220999999998 + ], + [ + -120.10884, + 37.166885999999998 + ], + [ + -120.107725, + 37.167230000000004 + ], + [ + -120.106489, + 37.167439999999999 + ], + [ + -120.10008500000001, + 37.170507000000001 + ], + [ + -120.096281, + 37.172128999999998 + ], + [ + -120.085251, + 37.173369999999998 + ], + [ + -120.080826, + 37.174433000000001 + ], + [ + -120.076733, + 37.176468999999997 + ], + [ + -120.071297, + 37.177562000000002 + ], + [ + -120.064685, + 37.177813999999998 + ], + [ + -120.052055, + 37.183107999999997 + ], + [ + -120.04871900000001, + 37.185836000000002 + ], + [ + -120.04644, + 37.187708000000001 + ], + [ + -120.013783, + 37.214229000000003 + ], + [ + -119.988494, + 37.234901999999998 + ], + [ + -119.982337, + 37.239871000000001 + ], + [ + -119.97869, + 37.242814000000003 + ], + [ + -119.974372, + 37.246298000000003 + ], + [ + -119.94647399999999, + 37.268737999999999 + ], + [ + -119.942402, + 37.272024000000002 + ], + [ + -119.941958, + 37.272382 + ], + [ + -119.932517, + 37.280002000000003 + ], + [ + -119.876374, + 37.325316000000001 + ], + [ + -119.86245599999999, + 37.336413 + ], + [ + -119.856011, + 37.341622000000001 + ], + [ + -119.782208, + 37.400852999999998 + ], + [ + -119.761802, + 37.417115000000003 + ], + [ + -119.747102, + 37.417748000000003 + ], + [ + -119.729856, + 37.417760000000001 + ], + [ + -119.70801400000001, + 37.417777000000001 + ], + [ + -119.706931, + 37.417777999999998 + ], + [ + -119.694124, + 37.417788999999999 + ], + [ + -119.690787, + 37.417791999999999 + ], + [ + -119.651191, + 37.417828 + ], + [ + -119.651177, + 37.446224999999998 + ], + [ + -119.651003, + 37.461353000000003 + ], + [ + -119.64573, + 37.461371999999997 + ], + [ + -119.636383, + 37.461359999999999 + ], + [ + -119.615489, + 37.461519000000003 + ], + [ + -119.615148, + 37.494681 + ], + [ + -119.585297, + 37.494694000000003 + ], + [ + -119.58422, + 37.494695999999998 + ], + [ + -119.583933, + 37.530375999999997 + ], + [ + -119.58363900000001, + 37.555795000000003 + ], + [ + -119.58358800000001, + 37.560178000000001 + ], + [ + -119.34533500000001, + 37.749927999999997 + ], + [ + -119.308995, + 37.777985999999999 + ], + [ + -119.308283, + 37.775728000000001 + ], + [ + -119.29338199999999, + 37.767828999999999 + ], + [ + -119.294082, + 37.759729 + ], + [ + -119.292782, + 37.758029000000001 + ], + [ + -119.28988200000001, + 37.756329000000001 + ], + [ + -119.28908199999999, + 37.755029 + ], + [ + -119.289582, + 37.749929000000002 + ], + [ + -119.290682, + 37.749329000000003 + ], + [ + -119.288381, + 37.74503 + ], + [ + -119.27688000000001, + 37.741630000000001 + ], + [ + -119.27178000000001, + 37.739229999999999 + ], + [ + -119.268979, + 37.739229999999999 + ] + ] + ] + } + }, + { + "attributes" : { + "OBJECTID" : 41, + "AREA" : 0.121590982187282, + "PERIMETER" : 2.0840705916534099, + "CO06_D00_" : 42, + "CO06_D00_I" : 41, + "STATE" : "06", + "COUNTY" : "081", + "NAME" : "San Mateo", + "LSAD" : "06", + "LSAD_TRANS" : "County", + "Shape_Length" : 2.0840705916534104, + "Shape_Area" : 0.12159098218727794 + }, + "geometry" : { + "rings" : [ + [ + [ + -122.426833, + 37.708373000000002 + ], + [ + -122.426366, + 37.708351 + ], + [ + -122.425882, + 37.708328000000002 + ], + [ + -122.425792, + 37.708323999999998 + ], + [ + -122.42556500000001, + 37.708314000000001 + ], + [ + -122.423782, + 37.708230999999998 + ], + [ + -122.42008199999999, + 37.708230999999998 + ], + [ + -122.416082, + 37.708331000000001 + ], + [ + -122.415182, + 37.708230999999998 + ], + [ + -122.41448200000001, + 37.708230999999998 + ], + [ + -122.414182, + 37.708230999999998 + ], + [ + -122.41399699999999, + 37.708252000000002 + ], + [ + -122.413282, + 37.708331000000001 + ], + [ + -122.41037, + 37.708283000000002 + ], + [ + -122.405582, + 37.708230999999998 + ], + [ + -122.40545299999999, + 37.708244999999998 + ], + [ + -122.405226, + 37.708271000000003 + ], + [ + -122.395782, + 37.708331000000001 + ], + [ + -122.39518200000001, + 37.708331000000001 + ], + [ + -122.393782, + 37.708230999999998 + ], + [ + -122.39268199999999, + 37.708331000000001 + ], + [ + -122.39137380314342, + 37.708331000000001 + ], + [ + -122.39318952871757, + 37.707530537249347 + ], + [ + -122.38762624308495, + 37.679059601210412 + ], + [ + -122.38024645718799, + 37.66973300156554 + ], + [ + -122.3742907217103, + 37.662206121901704 + ], + [ + -122.37559972636403, + 37.652388561631916 + ], + [ + -122.37789049084977, + 37.650425041967907 + ], + [ + -122.38738080629783, + 37.648461534987312 + ], + [ + -122.38607178896069, + 37.637662214885516 + ], + [ + -122.38173395968786, + 37.635252306917074 + ], + [ + -122.36545492127252, + 37.626208386115145 + ], + [ + -122.35531010349757, + 37.615736317176776 + ], + [ + -122.35858262781531, + 37.611154788205312 + ], + [ + -122.37036369506571, + 37.614427309352195 + ], + [ + -122.37330896187829, + 37.613772807025327 + ], + [ + -122.37787850747409, + 37.606632904484457 + ], + [ + -122.37854500586005, + 37.6055915025727 + ], + [ + -122.37565802461997, + 37.603529374257249 + ], + [ + -122.36021887729076, + 37.592501417985169 + ], + [ + -122.35906780870063, + 37.59245714641164 + ], + [ + -122.33389686552938, + 37.591489039670769 + ], + [ + -122.31767608652702, + 37.590865167716998 + ], + [ + -122.315385328383, + 37.587265398577543 + ], + [ + -122.31571257954643, + 37.58366562943808 + ], + [ + -122.30589501293493, + 37.575484318643738 + ], + [ + -122.30366927743523, + 37.575349425305902 + ], + [ + -122.2970124077065, + 37.574945977826467 + ], + [ + -122.2882781810038, + 37.574416629665599 + ], + [ + -122.2630243886212, + 37.572886093648201 + ], + [ + -122.26269773886945, + 37.572866296652862 + ], + [ + -122.2621145554226, + 37.572512851807538 + ], + [ + -122.25696363190799, + 37.569391077047378 + ], + [ + -122.25189841876764, + 37.566321248017388 + ], + [ + -122.25092954886826, + 37.5652681305491 + ], + [ + -122.24955669534106, + 37.563775901270624 + ], + [ + -122.24437161030016, + 37.558139949906462 + ], + [ + -122.24285240228596, + 37.55714916243285 + ], + [ + -122.24004767486696, + 37.555319993044812 + ], + [ + -122.23448271266635, + 37.551690671195587 + ], + [ + -122.22518463469684, + 37.545626709516419 + ], + [ + -122.22327315046184, + 37.544380089818297 + ], + [ + -122.21426441448051, + 37.538504829366879 + ], + [ + -122.19659281360492, + 37.537195812029729 + ], + [ + -122.19495655144605, + 37.522469471625044 + ], + [ + -122.16844912476586, + 37.504143349397474 + ], + [ + -122.16312267476584, + 37.502914167712092 + ], + [ + -122.15568629768346, + 37.501198079414024 + ], + [ + -122.14963214798911, + 37.502670715991172 + ], + [ + -122.14014183254103, + 37.507906759972919 + ], + [ + -122.13097876191469, + 37.503652482164888 + ], + [ + -122.12770625028037, + 37.50005270034201 + ], + [ + -122.12572334838893, + 37.500964836165032 + ], + [ + -122.11611629693846, + 37.505384084449311 + ], + [ + -122.10957399999999, + 37.497636999999997 + ], + [ + -122.081473, + 37.477837999999998 + ], + [ + -122.096762, + 37.461905000000002 + ], + [ + -122.096574, + 37.466138000000001 + ], + [ + -122.111548, + 37.466292000000003 + ], + [ + -122.113265, + 37.467529999999996 + ], + [ + -122.115374, + 37.466138000000001 + ], + [ + -122.11591300000001, + 37.465774000000003 + ], + [ + -122.12316, + 37.460889000000002 + ], + [ + -122.121588, + 37.454749 + ], + [ + -122.123831, + 37.452750000000002 + ], + [ + -122.126874, + 37.453738000000001 + ], + [ + -122.127697, + 37.452469000000001 + ], + [ + -122.136774, + 37.454438000000003 + ], + [ + -122.140235, + 37.456232 + ], + [ + -122.14077, + 37.456257000000001 + ], + [ + -122.14113999999999, + 37.456673000000002 + ], + [ + -122.14105600000001, + 37.457151000000003 + ], + [ + -122.144975, + 37.458238000000001 + ], + [ + -122.146384, + 37.456702 + ], + [ + -122.152475, + 37.457737999999999 + ], + [ + -122.15351699999999, + 37.456600000000002 + ], + [ + -122.15434999999999, + 37.455801999999998 + ], + [ + -122.15561599999999, + 37.455634000000003 + ], + [ + -122.15544800000001, + 37.454284000000001 + ], + [ + -122.156791, + 37.454177000000001 + ], + [ + -122.15747500000001, + 37.453538000000002 + ], + [ + -122.158475, + 37.453637999999998 + ], + [ + -122.159278, + 37.454627000000002 + ], + [ + -122.16025500000001, + 37.454574000000001 + ], + [ + -122.160777, + 37.453625000000002 + ], + [ + -122.162375, + 37.453938000000001 + ], + [ + -122.162988, + 37.453125999999997 + ], + [ + -122.168975, + 37.447538000000002 + ], + [ + -122.169239, + 37.447847000000003 + ], + [ + -122.16998, + 37.447608000000002 + ], + [ + -122.17077500000001, + 37.447437999999998 + ], + [ + -122.175275, + 37.443238000000001 + ], + [ + -122.17622900000001, + 37.441729000000002 + ], + [ + -122.17662900000001, + 37.441733999999997 + ], + [ + -122.176953, + 37.441423999999998 + ], + [ + -122.179975, + 37.439638000000002 + ], + [ + -122.180475, + 37.439138 + ], + [ + -122.18197600000001, + 37.439537999999999 + ], + [ + -122.182518, + 37.439025000000001 + ], + [ + -122.18367600000001, + 37.435037999999999 + ], + [ + -122.184776, + 37.433838999999999 + ], + [ + -122.18797600000001, + 37.433039000000001 + ], + [ + -122.190361, + 37.431297000000001 + ], + [ + -122.190076, + 37.428939 + ], + [ + -122.189835, + 37.428189000000003 + ], + [ + -122.19097600000001, + 37.424138999999997 + ], + [ + -122.190076, + 37.423338999999999 + ], + [ + -122.188276, + 37.421939000000002 + ], + [ + -122.18717599999999, + 37.416238999999997 + ], + [ + -122.18757600000001, + 37.414738999999997 + ], + [ + -122.19187599999999, + 37.413839000000003 + ], + [ + -122.193076, + 37.412539000000002 + ], + [ + -122.194481, + 37.407775999999998 + ], + [ + -122.192635, + 37.403815999999999 + ], + [ + -122.191643, + 37.398513999999999 + ], + [ + -122.19187599999999, + 37.397539000000002 + ], + [ + -122.190102, + 37.394005 + ], + [ + -122.18982699999999, + 37.391022999999997 + ], + [ + -122.190476, + 37.384839999999997 + ], + [ + -122.193076, + 37.382440000000003 + ], + [ + -122.199089, + 37.374496999999998 + ], + [ + -122.199196, + 37.372658999999999 + ], + [ + -122.200776, + 37.371040000000001 + ], + [ + -122.202476, + 37.363439999999997 + ], + [ + -122.20227800000001, + 37.360528000000002 + ], + [ + -122.20117999999999, + 37.357590999999999 + ], + [ + -122.197176, + 37.352739999999997 + ], + [ + -122.195351, + 37.347506000000003 + ], + [ + -122.193489, + 37.345041000000002 + ], + [ + -122.179401, + 37.329967000000003 + ], + [ + -122.17507500000001, + 37.325741000000001 + ], + [ + -122.17928000000001, + 37.326194999999998 + ], + [ + -122.179391, + 37.326206999999997 + ], + [ + -122.18047300000001, + 37.326219999999999 + ], + [ + -122.18246600000001, + 37.326479999999997 + ], + [ + -122.18377599999999, + 37.325840999999997 + ], + [ + -122.18460899999999, + 37.325074999999998 + ], + [ + -122.185458, + 37.323737000000001 + ], + [ + -122.18837600000001, + 37.321041000000001 + ], + [ + -122.192576, + 37.318741000000003 + ], + [ + -122.19067200000001, + 37.317867 + ], + [ + -122.188576, + 37.316541000000001 + ], + [ + -122.186348, + 37.313836999999999 + ], + [ + -122.185143, + 37.311092000000002 + ], + [ + -122.18232, + 37.309528 + ], + [ + -122.18116000000001, + 37.309795000000001 + ], + [ + -122.179711, + 37.311312999999998 + ], + [ + -122.17699, + 37.312626000000002 + ], + [ + -122.175875, + 37.312741000000003 + ], + [ + -122.17427499999999, + 37.312742 + ], + [ + -122.168775, + 37.309542 + ], + [ + -122.168235, + 37.307803 + ], + [ + -122.16658700000001, + 37.305613999999998 + ], + [ + -122.16267499999999, + 37.304442000000002 + ], + [ + -122.162879, + 37.300960000000003 + ], + [ + -122.162775, + 37.300803000000002 + ], + [ + -122.16574799999999, + 37.295856000000001 + ], + [ + -122.16573699999999, + 37.294871999999998 + ], + [ + -122.16564099999999, + 37.294863999999997 + ], + [ + -122.163875, + 37.294142000000001 + ], + [ + -122.16318, + 37.293844999999997 + ], + [ + -122.163067, + 37.294020000000003 + ], + [ + -122.163372, + 37.294199999999996 + ], + [ + -122.16231000000001, + 37.293694000000002 + ], + [ + -122.16184199999999, + 37.293536000000003 + ], + [ + -122.16094200000001, + 37.293393999999999 + ], + [ + -122.159645, + 37.293596999999998 + ], + [ + -122.158226, + 37.292890999999997 + ], + [ + -122.158068, + 37.292834999999997 + ], + [ + -122.155175, + 37.291142000000001 + ], + [ + -122.153983, + 37.290171999999998 + ], + [ + -122.15248699999999, + 37.286340000000003 + ], + [ + -122.152675, + 37.247642999999997 + ], + [ + -122.152664, + 37.244418000000003 + ], + [ + -122.15237500000001, + 37.241143000000001 + ], + [ + -122.151454, + 37.229936000000002 + ], + [ + -122.151668, + 37.227837999999998 + ], + [ + -122.15287499999999, + 37.220044000000001 + ], + [ + -122.15285299999999, + 37.216068 + ], + [ + -122.155034, + 37.215443999999998 + ], + [ + -122.155175, + 37.215443999999998 + ], + [ + -122.15734399999999, + 37.215443999999998 + ], + [ + -122.158175, + 37.215443999999998 + ], + [ + -122.180071, + 37.215285999999999 + ], + [ + -122.18091800000001, + 37.215209999999999 + ], + [ + -122.181017, + 37.215217000000003 + ], + [ + -122.181072, + 37.215254000000002 + ], + [ + -122.215239, + 37.215226000000001 + ], + [ + -122.217263, + 37.215155000000003 + ], + [ + -122.22115700000001, + 37.215074000000001 + ], + [ + -122.221447, + 37.215096000000003 + ], + [ + -122.22266, + 37.215187999999998 + ], + [ + -122.223265, + 37.215234000000002 + ], + [ + -122.224104, + 37.215119999999999 + ], + [ + -122.22559800000001, + 37.215300999999997 + ], + [ + -122.225859, + 37.215333000000001 + ], + [ + -122.230864, + 37.215234000000002 + ], + [ + -122.24257799999999, + 37.215133999999999 + ], + [ + -122.24258399999999, + 37.207681000000001 + ], + [ + -122.242659, + 37.200319 + ], + [ + -122.242653, + 37.199452999999998 + ], + [ + -122.24258399999999, + 37.193460000000002 + ], + [ + -122.24560099999999, + 37.189895 + ], + [ + -122.24777899999999, + 37.189844999999998 + ], + [ + -122.26888099999999, + 37.189843000000003 + ], + [ + -122.288668, + 37.186745999999999 + ], + [ + -122.317682, + 37.186945000000001 + ], + [ + -122.316063, + 37.175525999999998 + ], + [ + -122.31178199999999, + 37.147545999999998 + ], + [ + -122.29318000000001, + 37.119348000000002 + ], + [ + -122.29099100000001, + 37.115960999999999 + ], + [ + -122.289382, + 37.113447000000001 + ], + [ + -122.2930135271027, + 37.107345768907898 + ], + [ + -122.306139, + 37.116382999999999 + ], + [ + -122.313907, + 37.118161000000001 + ], + [ + -122.322971, + 37.115459999999999 + ], + [ + -122.33046299999999, + 37.115338000000001 + ], + [ + -122.33707099999999, + 37.117381999999999 + ], + [ + -122.33885600000001, + 37.120854000000001 + ], + [ + -122.337085, + 37.130794999999999 + ], + [ + -122.337833, + 37.135936000000001 + ], + [ + -122.34402900000001, + 37.144098999999997 + ], + [ + -122.359791, + 37.155574000000001 + ], + [ + -122.36179, + 37.163592999999999 + ], + [ + -122.367085, + 37.172817000000002 + ], + [ + -122.37927000000001, + 37.181128000000001 + ], + [ + -122.39059899999999, + 37.182988000000002 + ], + [ + -122.397065, + 37.187249000000001 + ], + [ + -122.405073, + 37.195791 + ], + [ + -122.40718099999999, + 37.219465 + ], + [ + -122.40898199999999, + 37.225257999999997 + ], + [ + -122.41582200000001, + 37.232838999999998 + ], + [ + -122.419113, + 37.24147 + ], + [ + -122.418452, + 37.248520999999997 + ], + [ + -122.411686, + 37.265844000000001 + ], + [ + -122.4070406751253, + 37.297744467500571 + ], + [ + -122.401323, + 37.337009000000002 + ], + [ + -122.40085000000001, + 37.359225000000002 + ], + [ + -122.40925799999999, + 37.374805000000002 + ], + [ + -122.40937458718714, + 37.374952412812846 + ], + [ + -122.41680671480934, + 37.384349593282955 + ], + [ + -122.41811969413867, + 37.38600972404744 + ], + [ + -122.423286, + 37.392541999999999 + ], + [ + -122.43846845834331, + 37.42483960843299 + ], + [ + -122.43846911249074, + 37.424840999999297 + ], + [ + -122.443687, + 37.435941 + ], + [ + -122.445987, + 37.461540999999997 + ], + [ + -122.4473033850729, + 37.465640999999998 + ], + [ + -122.45208700000001, + 37.480539999999998 + ], + [ + -122.46191669453539, + 37.491488840188779 + ], + [ + -122.467888, + 37.498139999999999 + ], + [ + -122.472388, + 37.500540000000001 + ], + [ + -122.47475410134543, + 37.499506101345432 + ], + [ + -122.47475630690595, + 37.499505137597168 + ], + [ + -122.47527565294777, + 37.499278202611933 + ], + [ + -122.48348503393767, + 37.4956910071705 + ], + [ + -122.485888, + 37.494641000000001 + ], + [ + -122.49378900000001, + 37.492341000000003 + ], + [ + -122.499289, + 37.495341000000003 + ], + [ + -122.50108899999999, + 37.498030551724128 + ], + [ + -122.50236756265949, + 37.499940974171494 + ], + [ + -122.5076736864394, + 37.507869359927462 + ], + [ + -122.51064232170769, + 37.512305081096457 + ], + [ + -122.516689, + 37.521340000000002 + ], + [ + -122.51731895527642, + 37.524875635064127 + ], + [ + -122.519533, + 37.537301999999997 + ], + [ + -122.51937279220002, + 37.537717593399584 + ], + [ + -122.516589, + 37.544938999999999 + ], + [ + -122.51478899999999, + 37.546138999999997 + ], + [ + -122.513688, + 37.552239 + ], + [ + -122.51808800000001, + 37.576138 + ], + [ + -122.51742421461702, + 37.586819713948735 + ], + [ + -122.51718700000001, + 37.590637000000001 + ], + [ + -122.51194180870982, + 37.593624578103395 + ], + [ + -122.50761219768485, + 37.59609065615065 + ], + [ + -122.501386, + 37.599637000000001 + ], + [ + -122.49727887475829, + 37.610796773564381 + ], + [ + -122.4971652335942, + 37.611105556370902 + ], + [ + -122.496786, + 37.612136 + ], + [ + -122.49529376058486, + 37.629759452463446 + ], + [ + -122.494085, + 37.644035000000002 + ], + [ + -122.4953776124503, + 37.664340365938443 + ], + [ + -122.4954298683642, + 37.665161242647365 + ], + [ + -122.49561473799635, + 37.668065319217895 + ], + [ + -122.49678400000001, + 37.686433000000001 + ], + [ + -122.50052832680024, + 37.700831999999522 + ], + [ + -122.5018866401484, + 37.706055463373026 + ], + [ + -122.5024267151224, + 37.708132349276738 + ], + [ + -122.48608299999999, + 37.708232000000002 + ], + [ + -122.485383, + 37.708232000000002 + ], + [ + -122.481083, + 37.708232000000002 + ], + [ + -122.47148300000001, + 37.708331999999999 + ], + [ + -122.470883, + 37.708232000000002 + ], + [ + -122.46888300000001, + 37.708232000000002 + ], + [ + -122.467883, + 37.708232000000002 + ], + [ + -122.46718300000001, + 37.708232000000002 + ], + [ + -122.466283, + 37.708131999999999 + ], + [ + -122.461383, + 37.708232000000002 + ], + [ + -122.45828299999999, + 37.708232000000002 + ], + [ + -122.45398299999999, + 37.708232000000002 + ], + [ + -122.45218300000001, + 37.708131999999999 + ], + [ + -122.44928299999999, + 37.708232000000002 + ], + [ + -122.44713400000001, + 37.708232000000002 + ], + [ + -122.446983, + 37.708232000000002 + ], + [ + -122.442082, + 37.708232000000002 + ], + [ + -122.441282, + 37.708331999999999 + ], + [ + -122.440782, + 37.708331999999999 + ], + [ + -122.440082, + 37.708331999999999 + ], + [ + -122.439144, + 37.708278 + ], + [ + -122.435382, + 37.708131999999999 + ], + [ + -122.43338199999999, + 37.708232000000002 + ], + [ + -122.430882, + 37.708230999999998 + ], + [ + -122.430027, + 37.708280999999999 + ], + [ + -122.428082, + 37.708430999999997 + ], + [ + -122.426833, + 37.708373000000002 + ] + ] + ] + } + } + ] +} diff --git a/plugin/trino-hive/src/test/resources/esri/counties_expected.txt b/plugin/trino-hive/src/test/resources/esri/counties_expected.txt new file mode 100644 index 000000000000..43bb687594fd --- /dev/null +++ b/plugin/trino-hive/src/test/resources/esri/counties_expected.txt @@ -0,0 +1,3 @@ +San Francisco X'00 00 00 00 03 05 00 00 00 45 d6 1a 4a ed a0 5e c0 23 a0 c2 11 a4 da 42 40 03 d9 6b 8d d5 96 5e c0 0b 12 b8 4b dd e7 42 40 01 00 00 00 5f 00 00 00 00 00 00 00 98 85 61 c2 27 a0 5e c0 0e b1 b0 14 a4 da 42 40 b8 3e ac 37 6a a0 5e c0 fa 0c a8 37 a3 dc 42 40 89 2b 4a 45 80 a0 5e c0 f1 5d 82 53 1f de 42 40 16 0f 22 ef 99 a0 5e c0 b5 21 54 a9 d9 df 42 40 f8 27 0e 02 9a a0 5e c0 aa fd 78 ef da df 42 40 80 24 54 6a b6 a0 5e c0 f0 6d 60 90 c4 e1 42 40 e9 46 58 54 c4 a0 5e c0 74 7b 49 63 b4 e2 42 40 45 d6 1a 4a ed a0 5e c0 5a 62 65 34 f2 e3 42 40 74 04 8f 76 a1 a0 5e c0 ea 1b dd ea 6d e4 42 40 df 15 c1 ff 56 a0 5e c0 ad 85 59 68 e7 e4 42 40 11 bc 16 4e 8d 9f 5e c0 42 d7 a6 f9 da e4 42 40 48 70 23 65 8b 9f 5e c0 f5 48 83 db da e4 42 40 fa 7c 94 11 17 9f 5e c0 75 22 c1 54 33 e5 42 40 82 71 70 e9 98 9e 5e c0 2e 57 3f 36 c9 e7 42 40 db 70 00 fd 19 9e 5e c0 a8 b4 4e 8b 82 e7 42 40 ac c4 da c6 ae 9d 5e c0 7e 89 84 dd fe e6 42 40 6a 10 d3 53 b0 9c 5e c0 7f 50 d4 ec 53 e7 42 40 48 a2 32 d2 57 9c 5e c0 d9 d0 0a 83 71 e7 42 40 81 76 fc a3 42 9b 5e c0 6c 91 a5 2b ce e7 42 40 0a e9 f0 9c 26 9b 5e c0 ef 54 df 91 cf e7 42 40 38 e6 82 07 e6 9a 5e c0 c4 94 54 cb d2 e7 42 40 d8 41 b2 b0 13 9a 5e c0 0b 12 b8 4b dd e7 42 40 44 d8 6a fe 93 99 5e c0 e5 a0 5a f4 3d e7 42 40 b3 c5 56 1a 7b 99 5e c0 e2 96 30 e5 1e e7 42 40 5f 00 c7 b4 3d 99 5e c0 db ff e0 11 90 e6 42 40 ac 33 5a c6 f2 98 5e c0 17 28 5b c2 e1 e5 42 40 b4 b9 82 33 de 98 5e c0 05 00 41 e6 b1 e5 42 40 20 c3 41 86 ad 98 5e c0 79 87 ff a9 40 e5 42 40 f0 63 e4 20 a9 98 5e c0 20 37 18 70 36 e5 42 40 20 6b ad 80 94 98 5e c0 3f 2e 73 94 43 e4 42 40 1c 7c 4c 26 68 98 5e c0 a3 b6 d7 58 39 e2 42 40 fd a6 24 94 41 98 5e c0 2e a7 0d 32 73 e0 42 40 9e d8 9f 6f 1d 98 5e c0 90 b6 b2 a2 c9 de 42 40 38 18 35 8b 0e 98 5e c0 94 f3 91 49 1a de 42 40 d4 19 c0 fd 0d 98 5e c0 7a ba fd c7 13 de 42 40 e2 3f 95 0b e5 97 5e c0 6e 69 35 24 ee dd 42 40 e7 50 61 9d af 97 5e c0 f7 8b 74 06 bd dd 42 40 b6 f3 31 8a af 97 5e c0 8c ea 42 30 bd dd 42 40 07 58 22 df a7 97 5e c0 fb 56 f9 e5 cd dd 42 40 67 57 0f 57 88 97 5e c0 f4 7d db 9b 12 de 42 40 ad 7e 7a 2f 88 97 5e c0 e8 b2 5d 90 12 de 42 40 a0 36 12 ff 63 97 5e c0 90 46 b3 0e 08 de 42 40 03 d9 6b 8d d5 96 5e c0 4f d0 47 68 60 dd 42 40 30 d1 00 e5 26 97 5e c0 ce b2 9a 6e 85 db 42 40 2a eb 6a d1 b4 97 5e c0 66 32 12 69 d9 db 42 40 07 c4 d0 60 03 98 5e c0 39 44 d9 24 94 db 42 40 c7 40 b5 44 0c 99 5e c0 1a df 17 97 aa da 42 40 95 ee ae b3 21 99 5e c0 1a df 17 97 aa da 42 40 39 f0 6a b9 33 99 5e c0 b6 81 3b 50 a7 da 42 40 f2 7d 71 a9 4a 99 5e c0 1a df 17 97 aa da 42 40 1c 96 06 7e 54 99 5e c0 1a df 17 97 aa da 42 40 4a 5f 08 39 ef 99 5e c0 45 da c6 9f a8 da 42 40 2a c4 23 f1 f2 99 5e c0 1b 47 ac c5 a7 da 42 40 29 76 34 0e f5 99 5e c0 b6 81 3b 50 a7 da 42 40 9c 50 88 80 43 9a 5e c0 09 a8 70 04 a9 da 42 40 a1 81 58 36 73 9a 5e c0 1a df 17 97 aa da 42 40 18 e9 45 ed 7e 9a 5e c0 ce a9 64 00 a8 da 42 40 e1 25 38 f5 81 9a 5e c0 b6 81 3b 50 a7 da 42 40 f7 b1 82 df 86 9a 5e c0 b6 81 3b 50 a7 da 42 40 d3 f8 85 57 92 9a 5e c0 b6 81 3b 50 a7 da 42 40 13 9d 65 16 a1 9a 5e c0 1a df 17 97 aa da 42 40 d9 e8 9c 9f e2 9a 5e c0 b6 81 3b 50 a7 da 42 40 8b a8 89 3e 1f 9b 5e c0 b6 81 3b 50 a7 da 42 40 a0 54 fb 74 3c 9b 5e c0 44 a6 7c 08 aa da 42 40 80 b9 16 2d 40 9b 5e c0 67 7c 5f 5c aa da 42 40 20 7d 93 a6 41 9b 5e c0 a9 6b ed 7d aa da 42 40 16 88 9e 94 49 9b 5e c0 61 8b dd 3e ab da 42 40 a1 f6 5b 3b 51 9b 5e c0 49 2f 6a f7 ab da 42 40 67 80 0b b2 65 9b 5e c0 7d 3c f4 dd ad da 42 40 68 59 f7 8f 85 9b 5e c0 68 b0 a9 f3 a8 da 42 40 d8 9b 18 92 93 9b 5e c0 b6 81 3b 50 a7 da 42 40 34 2b db 87 bc 9b 5e c0 87 fd 9e 58 a7 da 42 40 18 d1 76 4c dd 9b 5e c0 23 a0 c2 11 a4 da 42 40 04 8f 6f ef 1a 9c 5e c0 f7 3c 7f da a8 da 42 40 bb 63 b1 4d 2a 9c 5e c0 ea 5a 7b 9f aa da 42 40 97 aa b4 c5 35 9c 5e c0 ea 5a 7b 9f aa da 42 40 10 94 db f6 3d 9c 5e c0 ea 5a 7b 9f aa da 42 40 9e 09 4d 12 4b 9c 5e c0 87 fd 9e 58 a7 da 42 40 8d b7 95 5e 9b 9c 5e c0 87 fd 9e 58 a7 da 42 40 80 bb ec d7 9d 9c 5e c0 87 fd 9e 58 a7 da 42 40 85 e9 7b 0d c1 9c 5e c0 87 fd 9e 58 a7 da 42 40 a9 33 f7 90 f0 9c 5e c0 23 a0 c2 11 a4 da 42 40 28 7c b6 0e 0e 9d 5e c0 87 fd 9e 58 a7 da 42 40 04 54 38 82 54 9d 5e c0 87 fd 9e 58 a7 da 42 40 8b fb 8f 4c 87 9d 5e c0 87 fd 9e 58 a7 da 42 40 92 eb a6 94 d7 9d 5e c0 23 a0 c2 11 a4 da 42 40 d2 8f 86 53 e6 9d 5e c0 87 fd 9e 58 a7 da 42 40 ae d6 89 cb f1 9d 5e c0 87 fd 9e 58 a7 da 42 40 a0 a9 d7 2d 02 9e 5e c0 87 fd 9e 58 a7 da 42 40 83 4f 73 f2 22 9e 5e c0 87 fd 9e 58 a7 da 42 40 ae 67 08 c7 2c 9e 5e c0 ea 5a 7b 9f aa da 42 40 57 ea 59 10 ca 9e 5e c0 87 fd 9e 58 a7 da 42 40 33 c2 db 83 10 9f 5e c0 87 fd 9e 58 a7 da 42 40 0f 09 df fb 1b 9f 5e c0 87 fd 9e 58 a7 da 42 40 98 85 61 c2 27 a0 5e c0 0e b1 b0 14 a4 da 42 40' +Madera X'00 00 00 00 03 05 00 00 00 84 d8 99 42 e7 22 5e c0 c0 79 71 e2 ab 61 42 40 fd 68 38 65 6e c1 5d c0 78 5f 95 0b 95 e3 42 40 01 00 00 00 30 03 00 00 00 00 00 00 e4 13 b2 f3 36 d1 5d c0 6d 1c b1 16 9f de 42 40 dd 23 9b ab e6 d0 5d c0 bf 9c d9 ae d0 dd 42 40 ba d9 1f 28 b7 d0 5d c0 f8 e1 20 21 ca dd 42 40 f6 d4 ea ab ab d0 5d c0 ea 23 f0 87 9f dd 42 40 17 f2 08 6e a4 d0 5d c0 31 40 a2 09 14 dd 42 40 e8 16 ba 12 81 d0 5d c0 7a 55 67 b5 c0 dc 42 40 28 2a 1b d6 54 d0 5d c0 32 e3 6d a5 d7 dc 42 40 bd 6d a6 42 3c d0 5d c0 07 ed d5 c7 43 dd 42 40 e1 26 a3 ca 30 d0 5d c0 23 d8 b8 fe 5d dd 42 40 fe 80 07 06 10 d0 5d c0 76 6c 04 e2 75 dd 42 40 34 9d 9d 0c 8e cf 5d c0 a9 4c 31 07 41 dd 42 40 7f de 54 a4 c2 ce 5d c0 cf f5 7d 38 48 dc 42 40 dd 95 5d 30 b8 ce 5d c0 22 fa b5 f5 d3 db 42 40 e3 df 67 5c 38 ce 5d c0 6e fc 89 ca 86 db 42 40 eb 02 5e 66 d8 cd 5d c0 24 29 e9 61 68 db 42 40 e2 73 27 d8 7f cd 5d c0 84 11 fb 04 50 dc 42 40 24 9b ab e6 39 cd 5d c0 c3 ba f1 ee c8 dc 42 40 06 83 6b ee e8 cc 5d c0 eb 74 20 eb a9 dd 42 40 8d 62 b9 a5 d5 cc 5d c0 50 a7 3c ba 11 de 42 40 39 63 98 13 b4 cc 5d c0 4b af cd c6 4a de 42 40 4f 1e 16 6a 4d cc 5d c0 41 45 d5 af 74 de 42 40 08 aa 46 af 06 cc 5d c0 b5 e0 45 5f 41 de 42 40 18 43 39 d1 ae cb 5d c0 26 8e 3c 10 59 de 42 40 52 ba f4 2f 49 cb 5d c0 e7 8c 28 ed 0d de 42 40 b1 fd 64 8c 0f cb 5d c0 c0 ec 9e 3c 2c de 42 40 3f 1c 24 44 f9 ca 5d c0 55 4c a5 9f 70 de 42 40 ee ec 2b 0f d2 ca 5d c0 88 da 36 8c 82 de 42 40 4e d2 fc 31 ad ca 5d c0 da e3 85 74 78 de 42 40 d6 c5 6d 34 80 ca 5d c0 ad 6a 49 47 39 de 42 40 44 88 2b 67 ef c9 5d c0 7f a6 5e b7 08 de 42 40 79 ad 84 ee 92 c9 5d c0 72 33 dc 80 cf dd 42 40 b3 cd 8d e9 09 c9 5d c0 3e 77 82 fd d7 dd 42 40 6b 10 e6 76 2f c8 5d c0 dd 7a 4d 0f 0a de 42 40 06 a0 51 ba f4 c7 5d c0 37 8c 82 e0 f1 dd 42 40 1d ca 50 15 53 c7 5d c0 2c 0e 67 7e 35 dd 42 40 51 a4 fb 39 05 c7 5d c0 1f a2 d1 1d c4 dc 42 40 e1 27 0e a0 df c6 5d c0 c6 dd 20 5a 2b dc 42 40 9f aa 42 03 b1 c6 5d c0 94 be 10 72 de db 42 40 cd 04 c3 b9 86 c6 5d c0 29 26 6f 80 99 db 42 40 89 28 26 6f 80 c6 5d c0 8b 6d 52 d1 58 db 42 40 5b d2 51 0e 66 c6 5d c0 5e 11 fc 6f 25 db 42 40 10 5d 50 df 32 c6 5d c0 67 0e 49 2d 94 da 42 40 11 1a c1 c6 f5 c5 5d c0 92 ca 14 73 10 da 42 40 e9 49 99 d4 d0 c5 5d c0 73 be d8 7b f1 d9 42 40 9e 42 ae d4 b3 c4 5d c0 bc 96 90 0f 7a d8 42 40 8d 80 0a 47 90 c4 5d c0 5b 5b 78 5e 2a d8 42 40 0c 3c f7 1e 2e c4 5d c0 19 3a 76 50 89 d7 42 40 99 84 0b 79 04 c4 5d c0 14 3f c6 dc b5 d6 42 40 7e 3b 89 08 ff c3 5d c0 fb 3c 46 79 e6 d5 42 40 0a a1 83 2e e1 c3 5d c0 e2 ca d9 3b a3 d5 42 40 54 e6 e6 1b d1 c3 5d c0 c6 a6 95 42 20 d5 42 40 b3 25 ab 22 dc c3 5d c0 cf dc 43 c2 f7 d4 42 40 eb 36 a8 fd d6 c3 5d c0 c2 2f f5 f3 a6 d4 42 40 33 34 9e 08 e2 c3 5d c0 77 4c dd 95 5d d4 42 40 9c 1a 68 3e e7 c3 5d c0 ec f7 c4 3a 55 d4 42 40 f1 82 88 d4 b4 c3 5d c0 46 45 9c 4e b2 d3 42 40 2d 0a bb 28 7a c3 5d c0 d8 d6 4f ff 59 d3 42 40 96 21 8e 75 71 c3 5d c0 38 2c 0d fc a8 d2 42 40 2a 6f 47 38 2d c3 5d c0 56 9e 40 d8 29 d2 42 40 16 2f 16 86 c8 c2 5d c0 ac ad d8 5f 76 d1 42 40 0d e3 6e 10 ad c2 5d c0 98 15 8a 74 3f d1 42 40 96 ed 43 de 72 c2 5d c0 a7 02 ee 79 fe d0 42 40 bf 46 92 20 5c c2 5d c0 a5 49 29 e8 f6 d0 42 40 57 b5 a4 a3 1c c2 5d c0 9c a6 cf 0e b8 d0 42 40 7a 54 fc df 11 c2 5d c0 f8 36 fd d9 8f d0 42 40 83 18 e8 da 17 c2 5d c0 22 c4 95 b3 77 d0 42 40 d1 af ad 9f fe c1 5d c0 78 0c 8f fd 2c d0 42 40 c8 b7 77 0d fa c1 5d c0 8c 2c 99 63 79 cf 42 40 5f 25 1f bb 0b c2 5d c0 c1 8d 94 2d 92 ce 42 40 7e c6 85 03 21 c2 5d c0 d0 98 49 d4 0b ce 42 40 54 e5 7b 46 22 c2 5d c0 39 62 2d 3e 05 ce 42 40 19 1e fb 59 2c c2 5d c0 3d 28 28 45 2b cd 42 40 dd 7a 4d 0f 0a c2 5d c0 ea b1 2d 03 ce cc 42 40 0f d0 7d 39 b3 c1 5d c0 4f 5c 8e 57 20 cc 42 40 48 15 c5 ab ac c1 5d c0 41 9e 5d be f5 cb 42 40 57 42 77 49 9c c1 5d c0 6c 94 f5 9b 89 cb 42 40 fd 68 38 65 6e c1 5d c0 51 87 15 6e f9 ca 42 40 1e fe 9a ac 51 c8 5d c0 0a 12 db dd 03 c0 42 40 75 ad bd 4f 55 c8 5d c0 fb 76 12 11 fe bf 42 40 7f 50 17 29 94 cb 5d c0 8e 5c 37 a5 bc ba 42 40 c0 ae 26 4f 59 d2 5d c0 ae 0d 15 e3 fc af 42 40 6d 00 36 20 42 d3 5d c0 45 49 48 a4 6d ae 42 40 05 6a 31 78 98 d3 5d c0 90 f5 d4 ea ab ad 42 40 4b 93 52 d0 ed d3 5d c0 97 e1 3f dd 40 ad 42 40 ba f8 db 9e 20 d4 5d c0 eb c8 91 ce c0 ac 42 40 d1 57 90 66 2c d4 5d c0 59 db 14 8f 8b ac 42 40 f7 af ac 34 29 d4 5d c0 5d 1a bf f0 4a ac 42 40 6d 1b 46 41 f0 d3 5d c0 f0 c2 d6 6c e5 ab 42 40 7a fd 49 7c ee d3 5d c0 56 0e 2d b2 9d ab 42 40 7f bd c2 82 fb d3 5d c0 a3 20 78 7c 7b ab 42 40 3b c6 15 17 47 d4 5d c0 58 8e 90 81 3c ab 42 40 6d 59 be 2e c3 d4 5d c0 17 65 36 c8 24 ab 42 40 c6 f8 30 7b d9 d4 5d c0 df c2 ba f1 ee aa 42 40 9c 31 cc 09 da d4 5d c0 aa 0d 4e 44 bf aa 42 40 24 45 64 58 c5 d4 5d c0 e6 cb 0b b0 8f aa 42 40 15 03 24 9a 40 d4 5d c0 a1 f7 c6 10 00 aa 42 40 da 1b 7c 61 32 d4 5d c0 86 00 e0 d8 b3 a9 42 40 0f 46 ec 13 40 d4 5d c0 39 64 03 e9 62 a9 42 40 c5 54 fa 09 67 d4 5d c0 cb 12 9d 65 16 a9 42 40 08 8e cb b8 a9 d4 5d c0 bc 94 ba 64 1c a9 42 40 65 17 0c ae b9 d4 5d c0 da 90 7f 66 10 a9 42 40 4c c6 31 92 3d d5 5d c0 d9 24 3f e2 57 a8 42 40 7b a4 c1 6d 6d d5 5d c0 b8 ce bf 5d f6 a7 42 40 b4 00 6d ab 59 d5 5d c0 aa 81 e6 73 ee a6 42 40 52 49 9d 80 26 d5 5d c0 6e 6d e1 79 a9 a6 42 40 72 6d a8 18 e7 d4 5d c0 f8 16 d6 8d 77 a5 42 40 34 bc 59 83 f7 d4 5d c0 97 91 7a 4f e5 a4 42 40 6e a6 42 3c 12 d5 5d c0 71 ac 8b db 68 a4 42 40 2e 59 15 e1 26 d5 5d c0 17 9c c1 df 2f a4 42 40 cf d8 97 6c 3c d5 5d c0 ca 18 1f 66 2f a3 42 40 6c 7b bb 25 39 d5 5d c0 34 10 cb 66 0e a3 42 40 be 6a 65 c2 2f d5 5d c0 00 c9 74 e8 f4 a2 42 40 70 b4 e3 86 df d4 5d c0 28 0c ca 34 9a a2 42 40 68 ea 75 8b c0 d4 5d c0 1a db 6b 41 ef a1 42 40 1b 67 d3 11 c0 d4 5d c0 7e c6 85 03 21 a1 42 40 bc 41 b4 56 b4 d4 5d c0 28 d1 92 c7 d3 a0 42 40 b1 db 67 95 99 d4 5d c0 8a 76 15 52 7e a0 42 40 eb 71 df 6a 9d d4 5d c0 3d 10 59 a4 89 9f 42 40 d5 08 fd 4c bd d4 5d c0 e7 1a 66 68 3c 9f 42 40 bc 40 49 81 05 d5 5d c0 4c 4f 58 e2 01 9f 42 40 cf 87 67 09 32 d5 5d c0 34 d7 69 a4 a5 9e 42 40 9c 87 13 98 4e d5 5d c0 24 b5 50 32 39 9d 42 40 2b 31 cf 4a 5a d5 5d c0 f9 a3 a8 33 f7 9c 42 40 de ad 2c d1 59 d5 5d c0 eb 8e c5 36 a9 9c 42 40 31 b3 cf 63 94 d5 5d c0 9d 4a 06 80 2a 9c 42 40 f8 6d 88 f1 9a d5 5d c0 b5 fe 96 00 fc 9b 42 40 e8 f6 92 c6 68 d5 5d c0 97 e6 56 08 ab 9b 42 40 61 18 b0 e4 2a d5 5d c0 19 3a 76 50 89 9b 42 40 55 16 85 5d 14 d5 5d c0 d4 2b 65 19 e2 9a 42 40 cd b1 bc ab 1e d5 5d c0 3b 71 39 5e 81 9a 42 40 6f 65 89 ce 32 d5 5d c0 c7 7f 81 20 40 9a 42 40 8a ca 86 35 95 d5 5d c0 52 b9 89 5a 9a 99 42 40 6c 08 8e cb b8 d5 5d c0 cf 85 91 5e d4 98 42 40 3c 15 70 cf f3 d5 5d c0 34 9e 08 e2 3c 98 42 40 2d 96 22 f9 4a d6 5d c0 bd 6e 11 18 eb 97 42 40 20 b3 b3 e8 9d d6 5d c0 58 6f d4 0a d3 97 42 40 d0 61 be bc 00 d7 5d c0 75 73 f1 b7 3d 97 42 40 8b 8b a3 72 13 d7 5d c0 af 3e 1e fa ee 96 42 40 ce fc 6a 0e 10 d7 5d c0 b0 c7 44 4a b3 95 42 40 af 06 28 0d 35 d7 5d c0 f1 b7 3d 41 62 95 42 40 44 c1 8c 29 58 d7 5d c0 b2 0d dc 81 3a 95 42 40 23 2f 6b 62 81 d7 5d c0 4e 0e 9f 74 22 95 42 40 07 eb ff 1c e6 d7 5d c0 db 15 fa 60 19 95 42 40 95 2c 27 a1 f4 d7 5d c0 41 f2 ce a1 0c 95 42 40 38 48 88 f2 05 d8 5d c0 93 8d 07 5b ec 94 42 40 bf 61 a2 41 0a d8 5d c0 54 e3 a5 9b c4 94 42 40 c0 af 91 24 08 d8 5d c0 6f 2d 93 e1 78 94 42 40 6e 6c 76 a4 fa d7 5d c0 3d 7c 99 28 42 94 42 40 9b 20 ea 3e 00 d8 5d c0 51 15 53 e9 27 94 42 40 73 d7 12 f2 41 d8 5d c0 9e d0 eb 4f e2 93 42 40 1b 0d e0 2d 90 d8 5d c0 a8 3a e4 66 b8 93 42 40 43 55 4c a5 9f d8 5d c0 ca 51 80 28 98 93 42 40 03 08 1f 4a b4 d8 5d c0 a0 51 ba f4 2f 93 42 40 93 56 7c 43 e1 d8 5d c0 1e 8a 02 7d 22 93 42 40 da ca 4b fe 27 d9 5d c0 30 46 24 0a 2d 93 42 40 6c 97 36 1c 96 d9 5d c0 e9 65 14 cb 2d 93 42 40 4b ad f7 1b ed d9 5d c0 2e e7 52 5c 55 94 42 40 bc 94 ba 64 1c da 5d c0 88 48 4d bb 98 94 42 40 90 2f a1 82 c3 da 5d c0 18 42 ce fb ff 94 42 40 4a 5e 9d 63 40 db 5d c0 5b ea 20 af 07 95 42 40 0f 5f 26 8a 90 db 5d c0 51 4d 49 d6 e1 94 42 40 2d b2 9d ef a7 db 5d c0 96 23 64 20 cf 94 42 40 2b 4d 4a 41 b7 db 5d c0 ef e3 68 8e ac 94 42 40 5a b7 41 ed b7 db 5d c0 00 3c a2 42 75 93 42 40 5f 5d 15 a8 c5 db 5d c0 44 a5 11 33 fb 92 42 40 46 b6 f3 fd d4 db 5d c0 19 02 80 63 cf 92 42 40 3f 52 44 86 55 dc 5d c0 61 fa 5e 43 70 92 42 40 28 b7 ed 7b d4 dc 5d c0 cf d7 2c 97 8d 92 42 40 95 0e d6 ff 39 dd 5d c0 51 82 fe 42 8f 92 42 40 bf 80 5e b8 73 dd 5d c0 e2 e4 7e 87 a2 92 42 40 60 00 e1 43 89 dd 5d c0 02 66 be 83 9f 92 42 40 8a e4 2b 81 94 dd 5d c0 be a0 85 04 8c 92 42 40 3a b1 87 f6 b1 dd 5d c0 28 60 3b 18 b1 91 42 40 e1 95 24 cf f5 dd 5d c0 fe f3 34 60 90 90 42 40 d5 04 51 f7 01 de 5d c0 4c dd 95 5d 30 90 42 40 8f 72 30 9b 00 de 5d c0 ae d9 ca 4b fe 8f 42 40 cf 6b ec 12 d5 dd 5d c0 f8 16 d6 8d 77 8f 42 40 98 f6 cd fd d5 dd 5d c0 d7 68 39 d0 43 8f 42 40 92 ca 14 73 10 de 5d c0 6f 7f 2e 1a 32 8e 42 40 4b 58 1b 63 27 de 5d c0 93 1b 45 d6 1a 8e 42 40 d4 0b 3e cd c9 de 5d c0 0d e2 03 3b fe 8d 42 40 fb 1f 60 ad da de 5d c0 56 47 8e 74 06 8e 42 40 8f a6 7a 32 ff de 5d c0 b7 7c 24 25 3d 8e 42 40 65 ff 3c 0d 18 df 5d c0 60 3a ad db a0 8e 42 40 a6 62 63 5e 47 df 5d c0 7d 41 0b 09 18 8f 42 40 d3 87 2e a8 6f df 5d c0 2f 35 42 3f 53 8f 42 40 40 4b 57 b0 8d df 5d c0 29 25 04 ab ea 8f 42 40 54 e4 10 71 73 df 5d c0 ac c6 12 d6 c6 90 42 40 bc 59 83 f7 55 df 5d c0 39 99 b8 55 10 91 42 40 fe 5f 75 e4 48 df 5d c0 6e 88 f1 9a 57 91 42 40 ae 0c aa 0d 4e df 5d c0 90 bb 08 53 94 91 42 40 b3 7e 33 31 5d df 5d c0 99 f0 4b fd bc 91 42 40 a1 66 48 15 c5 df 5d c0 cf 69 16 68 77 92 42 40 7d 04 fe f0 f3 df 5d c0 f1 f4 4a 59 86 92 42 40 bb 80 97 19 36 e0 5d c0 4e 60 3a ad db 92 42 40 f8 51 0d fb 3d e0 5d c0 55 bf d2 f9 f0 92 42 40 d6 e6 ff 55 47 e0 5d c0 67 81 76 87 14 93 42 40 33 8d 26 17 63 e0 5d c0 fe 0e 45 81 3e 93 42 40 e1 79 a9 d8 98 e0 5d c0 f5 81 e4 9d 43 93 42 40 11 a8 fe 41 24 e1 5d c0 d6 ab c8 e8 80 92 42 40 70 b2 0d dc 81 e1 5d c0 23 66 f6 79 8c 90 42 40 0b 0d c4 b2 99 e1 5d c0 23 db f9 7e 6a 90 42 40 fb cb ee c9 c3 e1 5d c0 04 c8 d0 b1 83 90 42 40 c9 90 63 eb 19 e2 5d c0 5b ea 20 af 07 91 42 40 4c df 6b 08 8e e2 5d c0 19 75 ad bd 4f 91 42 40 f0 fa cc 59 9f e2 5d c0 e7 df 2e fb 75 91 42 40 47 c7 d5 c8 ae e2 5d c0 4c 53 04 38 bd 91 42 40 42 ec 4c a1 f3 e2 5d c0 eb 55 64 74 40 92 42 40 1a a6 b6 d4 41 e3 5d c0 a2 0c 55 31 95 92 42 40 35 29 05 dd 5e e3 5d c0 4b ae 62 f1 9b 92 42 40 8a 5b 05 31 d0 e3 5d c0 b2 9b 19 fd 68 92 42 40 cd 5a 0a 48 fb e3 5d c0 25 c9 73 7d 1f 92 42 40 6e a6 42 3c 12 e4 5d c0 36 b0 55 82 c5 91 42 40 a7 eb 89 ae 0b e4 5d c0 a0 8b 86 8c 47 91 42 40 bc 5d 2f 4d 11 e4 5d c0 1a 4e 99 9b 6f 90 42 40 3b 8f 8a ff 3b e4 5d c0 09 c2 15 50 a8 8f 42 40 12 a4 52 ec 68 e4 5d c0 e5 61 a1 d6 34 8f 42 40 12 a4 52 ec 68 e4 5d c0 06 f4 c2 9d 0b 8f 42 40 99 d4 d0 06 60 e4 5d c0 f1 2d ac 1b ef 8e 42 40 7c b5 a3 38 47 e4 5d c0 e5 45 26 e0 d7 8e 42 40 33 4f ae 29 90 e3 5d c0 03 42 eb e1 cb 8e 42 40 09 a6 9a 59 4b e3 5d c0 b4 b0 a7 1d fe 8e 42 40 da e4 f0 49 27 e3 5d c0 eb 8d 5a 61 fa 8e 42 40 5a 68 e7 34 0b e3 5d c0 1e a8 53 1e dd 8e 42 40 7b 49 63 b4 8e e2 5d c0 56 2a a8 a8 fa 8d 42 40 6b bb 09 be 69 e2 5d c0 36 e5 0a ef 72 8d 42 40 0a 9f ad 83 83 e2 5d c0 a8 71 6f 7e c3 8c 42 40 52 d6 6f 26 a6 e2 5d c0 3c 31 eb c5 50 8c 42 40 d2 52 79 3b c2 e2 5d c0 3f c7 47 8b 33 8c 42 40 b6 f8 14 00 e3 e2 5d c0 f7 e6 37 4c 34 8c 42 40 a9 f8 bf 23 2a e3 5d c0 d0 98 49 d4 0b 8c 42 40 2a 3a 92 cb 7f e3 5d c0 a3 02 27 db c0 8b 42 40 82 ff ad 64 c7 e3 5d c0 e6 03 02 9d 49 8b 42 40 46 0a 65 e1 eb e3 5d c0 cb 49 28 7d 21 8a 42 40 40 be 84 0a 0e e4 5d c0 ba 82 6d c4 93 89 42 40 2b 18 95 d4 09 e4 5d c0 23 f5 9e ca 69 89 42 40 e1 44 f4 6b eb e3 5d c0 4a 27 12 4c 35 89 42 40 db 36 8c 82 e0 e3 5d c0 ac 74 77 9d 0d 89 42 40 d2 c6 11 6b f1 e3 5d c0 6b d2 6d 89 5c 88 42 40 6d 55 12 d9 07 e4 5d c0 27 6b d4 43 34 88 42 40 7b a1 80 ed 60 e4 5d c0 0c 05 6c 07 23 88 42 40 d2 8b da fd 2a e5 5d c0 70 95 27 10 76 88 42 40 7a 8a 1c 22 6e e5 5d c0 f5 f2 3b 4d 66 88 42 40 e9 0f cd 3c b9 e5 5d c0 3e 5e 48 87 87 88 42 40 d6 3b dc 0e 0d e6 5d c0 5b cd 3a e3 fb 88 42 40 32 ff e8 9b 34 e6 5d c0 8b a8 89 3e 1f 89 42 40 78 b7 b2 44 67 e6 5d c0 0e a4 8b 4d 2b 89 42 40 ae 45 0b d0 b6 e6 5d c0 12 bf 62 0d 17 89 42 40 e6 76 2f f7 c9 e6 5d c0 5e ba 49 0c 02 89 42 40 8c d6 51 d5 04 e7 5d c0 0b 28 d4 d3 47 88 42 40 c6 89 af 76 14 e7 5d c0 67 44 69 6f f0 87 42 40 10 06 9e 7b 0f e7 5d c0 89 b3 22 6a a2 87 42 40 61 aa 99 b5 14 e7 5d c0 af 60 1b f1 64 87 42 40 9e 06 0c 92 3e e7 5d c0 a3 22 4e 27 d9 86 42 40 a7 b0 52 41 45 e7 5d c0 d4 9a e6 1d a7 86 42 40 f6 cf d3 80 41 e7 5d c0 9b ad bc e4 7f 86 42 40 3f 70 95 27 10 e7 5d c0 a3 3c f3 72 d8 85 42 40 2a 1b d6 54 16 e7 5d c0 ba d9 1f 28 b7 85 42 40 25 08 57 40 a1 e7 5d c0 21 07 25 cc b4 83 42 40 24 42 23 d8 b8 e7 5d c0 2e 8c f4 a2 76 83 42 40 81 b4 ff 01 d6 e7 5d c0 51 a3 90 64 56 83 42 40 5d 89 40 f5 0f e8 5d c0 84 bd 89 21 39 83 42 40 9a 5c 8c 81 75 e8 5d c0 d0 62 29 92 af 82 42 40 8a e4 2b 81 94 e8 5d c0 ef e3 68 8e ac 82 42 40 eb 74 20 eb a9 e8 5d c0 38 32 8f fc c1 82 42 40 a2 97 51 2c b7 e8 5d c0 e4 85 74 78 08 83 42 40 c2 32 36 74 b3 e8 5d c0 fe 0e 45 81 3e 83 42 40 98 35 b1 c0 57 e8 5d c0 d2 51 0e 66 13 84 42 40 b5 c6 a0 13 42 e8 5d c0 fd 68 38 65 6e 84 42 40 bd 6e 11 18 eb e8 5d c0 c5 ab ac 6d 8a 85 42 40 43 e1 b3 75 70 e9 5d c0 f2 b0 50 6b 9a 85 42 40 39 0c e6 af 90 e9 5d c0 b0 3c 48 4f 91 85 42 40 57 7c 43 e1 b3 e9 5d c0 71 00 fd be 7f 85 42 40 78 27 9f 1e db e9 5d c0 77 2c b6 49 45 85 42 40 04 af 96 3b 33 ea 5d c0 2c 49 9e eb fb 84 42 40 9a ef e0 27 0e ea 5d c0 ed 65 db 69 6b 82 42 40 cd ae 7b 2b 12 ea 5d c0 c3 d3 2b 65 19 82 42 40 4b 72 c0 ae 26 ea 5d c0 21 07 25 cc b4 81 42 40 c8 24 23 67 61 ea 5d c0 82 37 a4 51 81 81 42 40 0e 13 0d 52 f0 ea 5d c0 9a 94 82 6e 2f 81 42 40 67 63 25 e6 59 eb 5d c0 e4 10 71 73 2a 81 42 40 cf 9f 36 aa d3 eb 5d c0 99 b7 ea 3a 54 81 42 40 df 70 1f b9 35 ec 5d c0 13 7d 3e ca 88 81 42 40 83 31 22 51 68 ec 5d c0 25 5d 33 f9 66 81 42 40 ef c9 c3 42 ad ec 5d c0 e4 f3 8a a7 1e 81 42 40 f0 85 c9 54 c1 ec 5d c0 d9 05 83 6b ee 80 42 40 c9 ab 73 0c c8 ec 5d c0 4f cf bb b1 a0 80 42 40 91 b7 5c fd d8 ec 5d c0 9f 94 49 0d 6d 80 42 40 d1 3d eb 1a 2d ed 5d c0 7e 55 2e 54 fe 7f 42 40 d0 5f e8 11 a3 ed 5d c0 39 0e bc 5a ee 7e 42 40 90 32 e2 02 d0 ee 5d c0 0f 80 b8 ab 57 7d 42 40 84 0c e4 d9 e5 ee 5d c0 83 89 3f 8a 3a 7d 42 40 fc 17 08 02 64 ef 5d c0 da e4 f0 49 27 7c 42 40 43 1b 80 0d 88 ef 5d c0 de cb 7d 72 14 7a 42 40 6f 82 6f 9a 3e ef 5d c0 29 cd e6 71 18 7a 42 40 1d 3c 13 9a 24 ef 5d c0 9a 06 45 f3 00 7a 42 40 8e 21 00 38 f6 ee 5d c0 a6 d3 ba 0d 6a 79 42 40 7e 55 2e 54 fe ee 5d c0 e8 31 ca 33 2f 79 42 40 8c d9 92 55 11 ef 5d c0 c5 e7 4e b0 ff 78 42 40 8b fc fa 21 36 ef 5d c0 c4 0b 22 52 d3 78 42 40 d2 ff 72 2d 5a ef 5d c0 8b c6 da df d9 78 42 40 97 00 fc 53 aa ef 5d c0 7d 3d 5f b3 5c 78 42 40 83 50 de c7 d1 ef 5d c0 61 fb c9 18 1f 78 42 40 d8 11 87 6c 20 f0 5d c0 45 7f 68 e6 c9 77 42 40 26 e3 18 c9 1e f0 5d c0 e3 ff 8e a8 50 77 42 40 6d 55 12 d9 07 f0 5d c0 80 80 b5 6a d7 76 42 40 66 87 f8 87 2d f0 5d c0 d6 fd 63 21 3a 76 42 40 90 30 0c 58 72 f0 5d c0 c9 ae b4 8c d4 75 42 40 e5 f1 b4 fc c0 f0 5d c0 d7 db 66 2a c4 75 42 40 a9 87 68 74 07 f1 5d c0 2d 7b 12 d8 9c 75 42 40 cc 62 62 f3 71 f1 5d c0 02 63 7d 03 93 75 42 40 36 1f d7 86 8a f1 5d c0 3c 86 c7 7e 16 75 42 40 8b 4f 01 30 9e f1 5d c0 da 75 6f 45 62 74 42 40 1f d7 86 8a 71 f2 5d c0 c0 24 95 29 e6 72 42 40 d8 64 8d 7a 88 f2 5d c0 88 bd 50 c0 76 72 42 40 4a ef 1b 5f 7b f2 5d c0 ec 89 ae 0b 3f 72 42 40 35 63 d1 74 76 f2 5d c0 c1 e0 9a 3b fa 71 42 40 11 19 56 f1 46 f2 5d c0 18 5e 49 f2 5c 71 42 40 60 ea e7 4d 45 f2 5d c0 ef 92 38 2b a2 70 42 40 5f 7b 66 49 80 f2 5d c0 61 fb c9 18 1f 70 42 40 f8 70 c9 71 a7 f2 5d c0 61 fb c9 18 1f 70 42 40 1f 68 05 86 ac f2 5d c0 61 fb c9 18 1f 70 42 40 cb d9 3b a3 ad f2 5d c0 de 1c ae d5 1e 70 42 40 05 c4 24 5c c8 f2 5d c0 29 cd e6 71 18 70 42 40 9f e3 a3 c5 19 f3 5d c0 65 c6 db 4a af 6f 42 40 7a 19 c5 72 4b f3 5d c0 68 23 d7 4d 29 6f 42 40 80 9b c5 8b 85 f3 5d c0 77 2e 8c f4 a2 6e 42 40 14 97 e3 15 88 f3 5d c0 b0 73 d3 66 9c 6e 42 40 8c 11 89 42 cb f3 5d c0 3f 58 c6 86 6e 6e 42 40 af 5b 04 c6 fa f3 5d c0 95 d5 74 3d d1 6d 42 40 28 45 2b f7 02 f4 5d c0 33 c5 1c 04 1d 6d 42 40 d1 e6 38 b7 09 f4 5d c0 5c 3b 51 12 12 6d 42 40 84 65 6c e8 66 f4 5d c0 7b 15 19 1d 90 6c 42 40 c3 9a ca a2 b0 f4 5d c0 6c e8 66 7f a0 6c 42 40 2e ad 86 c4 3d f5 5d c0 b4 b0 a7 1d fe 6c 42 40 dc 82 a5 ba 80 f5 5d c0 4f b0 ff 3a 37 6d 42 40 aa 9c f6 94 9c f5 5d c0 eb e3 a1 ef 6e 6d 42 40 c6 87 d9 cb b6 f5 5d c0 b1 c0 57 74 eb 6d 42 40 31 44 4e 5f cf f5 5d c0 3f c7 47 8b 33 6e 42 40 78 47 c6 6a f3 f5 5d c0 6a df dc 5f 3d 6e 42 40 76 ff 58 88 0e f6 5d c0 0c 22 52 d3 2e 6e 42 40 d8 7f 9d 9b 36 f6 5d c0 23 dc 64 54 19 6e 42 40 df de 35 e8 4b f6 5d c0 b1 c0 57 74 eb 6d 42 40 1f 83 15 a7 5a f6 5d c0 5c ff ae cf 9c 6d 42 40 d0 42 02 46 97 f6 5d c0 7a 37 16 14 06 6d 42 40 50 8b c1 c3 b4 f6 5d c0 b3 7c 5d 86 ff 6c 42 40 82 02 ef e4 d3 f6 5d c0 96 22 f9 4a 20 6d 42 40 82 02 ef e4 d3 f6 5d c0 d4 d3 47 e0 0f 6d 42 40 a3 73 7e 8a e3 f6 5d c0 3f 90 bc 73 28 6d 42 40 25 95 29 e6 20 f7 5d c0 45 80 d3 bb 78 6d 42 40 3a b2 f2 cb 60 f7 5d c0 54 8b 88 62 f2 6c 42 40 64 ca 87 a0 6a f7 5d c0 e3 de fc 86 89 6c 42 40 30 0e 2e 1d 73 f7 5d c0 bf d4 cf 9b 8a 6c 42 40 d3 2f 11 6f 9d f7 5d c0 aa 99 b5 14 90 6c 42 40 21 01 a3 cb 9b f7 5d c0 38 a0 a5 2b d8 6c 42 40 e8 dd 58 50 18 f8 5d c0 b7 0a 62 a0 6b 6d 42 40 24 ee b1 f4 a1 f8 5d c0 1a 8a 3b de e4 6d 42 40 f2 d0 77 b7 b2 f8 5d c0 47 38 2d 78 d1 6d 42 40 f9 66 9b 1b d3 f8 5d c0 1b 68 3e e7 6e 6d 42 40 7c 2b 12 13 d4 f8 5d c0 6c b2 46 3d 44 6d 42 40 b2 f4 a1 0b ea f8 5d c0 71 76 6b 99 0c 6d 42 40 23 a1 2d e7 52 f9 5d c0 38 31 24 27 13 6d 42 40 ea 5b e6 74 59 f9 5d c0 29 04 72 89 23 6d 42 40 46 7c 27 66 bd f9 5d c0 d4 d3 47 e0 0f 6d 42 40 91 28 b4 ac fb f9 5d c0 38 a0 a5 2b d8 6c 42 40 df 8a c4 04 35 fa 5d c0 ab 08 37 19 55 6c 42 40 6d 91 b4 1b 7d fa 5d c0 00 a8 e2 c6 2d 6c 42 40 5e f5 80 79 c8 fa 5d c0 5b 44 14 93 37 6c 42 40 b3 b6 29 1e 17 fb 5d c0 13 63 99 7e 89 6c 42 40 62 c0 92 ab 58 fb 5d c0 cc f0 9f 6e a0 6c 42 40 4c c5 c6 bc 8e fb 5d c0 69 02 45 2c 62 6c 42 40 2f 6b 62 81 af fb 5d c0 14 41 9c 87 13 6c 42 40 b6 81 3b 50 a7 fb 5d c0 f8 55 b9 50 f9 6b 42 40 2f 6b 62 81 af fb 5d c0 a2 25 8f a7 e5 6b 42 40 c4 3f 6c e9 d1 fb 5d c0 32 e8 84 d0 41 6b 42 40 bd 71 52 98 f7 fb 5d c0 96 b4 e2 1b 0a 6b 42 40 e0 4c 4c 17 62 fc 5d c0 6b 0b cf 4b c5 6a 42 40 9b 1d a9 be f3 fc 5d c0 7e c7 f0 d8 cf 6a 42 40 7e 54 c3 7e 4f fd 5d c0 dd 94 f2 5a 09 6b 42 40 cf a0 a1 7f 82 fd 5d c0 a5 2c 43 1c eb 6a 42 40 54 3a 58 ff e7 fd 5d c0 ad 35 94 da 8b 6a 42 40 9b 20 ea 3e 00 fe 5d c0 cc b6 d3 d6 88 6a 42 40 74 b4 aa 25 1d fe 5d c0 1b 2a c6 f9 9b 6a 42 40 15 ff 77 44 85 fe 5d c0 86 8e 1d 54 e2 6a 42 40 8d 0b 07 42 b2 fe 5d c0 7f 16 4b 91 7c 6b 42 40 29 5f d0 42 02 ff 5d c0 99 82 35 ce a6 6b 42 40 ff b3 e6 c7 5f ff 5d c0 ea 07 75 91 42 6b 42 40 4d d9 e9 07 75 ff 5d c0 a1 f4 85 90 f3 6a 42 40 a7 58 35 08 73 ff 5d c0 ae 9d 28 09 89 6a 42 40 5a b8 ac c2 66 ff 5d c0 f7 8e 1a 13 62 6a 42 40 9c f8 6a 47 71 ff 5d c0 36 06 9d 10 3a 6a 42 40 63 9c bf 09 85 ff 5d c0 0e a1 4a cd 1e 6a 42 40 90 d7 83 49 f1 ff 5d c0 8f 52 09 4f e8 69 42 40 8d 43 fd 2e 6c 00 5e c0 71 39 5e 81 e8 69 42 40 19 20 d1 04 8a 00 5e c0 85 7a fa 08 fc 69 42 40 49 d7 4c be d9 00 5e c0 87 33 bf 9a 03 6a 42 40 86 93 34 7f 4c 01 5e c0 5d e2 c8 03 91 69 42 40 a8 52 b3 07 5a 01 5e c0 64 93 fc 88 5f 69 42 40 00 53 06 0e 68 01 5e c0 2d 26 36 1f d7 68 42 40 43 38 66 d9 93 01 5e c0 e5 7a db 4c 85 68 42 40 ba 9f 53 90 9f 01 5e c0 61 dd 78 77 64 68 42 40 5c 56 61 33 c0 01 5e c0 d5 b2 b5 be 48 68 42 40 50 51 f5 2b 9d 02 5e c0 79 e8 bb 5b 59 68 42 40 c7 66 47 aa ef 02 5e c0 76 fa 41 5d a4 68 42 40 5c e3 33 d9 3f 03 5e c0 3f e1 ec d6 32 69 42 40 ec f9 9a e5 b2 03 5e c0 4b e6 58 de 55 69 42 40 db a7 e3 31 03 04 5e c0 d8 0b 05 6c 07 69 42 40 6c 77 0f d0 7d 04 5e c0 55 17 f0 32 c3 68 42 40 75 af 93 fa b2 04 5e c0 86 c8 e9 eb f9 68 42 40 88 10 57 ce de 04 5e c0 d7 16 9e 97 8a 69 42 40 9e 95 b4 e2 1b 05 5e c0 0b 5e f4 15 a4 69 42 40 b5 70 59 85 cd 05 5e c0 a6 42 3c 12 2f 69 42 40 b9 a7 ab 3b 16 06 5e c0 78 7f bc 57 ad 68 42 40 0e be 30 99 2a 06 5e c0 40 34 f3 e4 9a 68 42 40 60 ea e7 4d 45 06 5e c0 fa 2a f9 d8 5d 68 42 40 82 56 60 c8 ea 06 5e c0 a2 7c 41 0b 09 68 42 40 c3 9f e1 cd 1a 07 5e c0 a3 01 bc 05 12 68 42 40 05 6e dd cd 53 07 5e c0 e5 7b 46 22 34 68 42 40 5b 61 fa 5e 43 08 5e c0 59 dd ea 39 e9 67 42 40 a7 06 9a cf b9 08 5e c0 26 56 46 23 9f 67 42 40 6c af 05 bd 37 09 5e c0 9b 54 34 d6 fe 66 42 40 f2 ee c8 58 6d 09 5e c0 df a5 d4 25 e3 66 42 40 a6 d2 4f 38 bb 09 5e c0 c9 e6 aa 79 8e 66 42 40 b3 96 02 d2 fe 09 5e c0 6b 47 71 8e 3a 66 42 40 24 0d 6e 6b 0b 0a 5e c0 65 ff 3c 0d 18 66 42 40 de 05 4a 0a 2c 0a 5e c0 6e dd cd 53 1d 66 42 40 9f ae ee 58 6c 0a 5e c0 8e 06 f0 16 48 66 42 40 8c 2c 99 63 79 0a 5e c0 85 d0 41 97 70 66 42 40 15 e1 26 a3 ca 0a 5e c0 3e ae 0d 15 e3 66 42 40 57 41 0c 74 ed 0a 5e c0 c1 8c 29 58 e3 66 42 40 94 32 a9 a1 0d 0b 5e c0 b4 1f 29 22 c3 66 42 40 e5 0d 30 f3 1d 0b 5e c0 4f ac 53 e5 7b 66 42 40 72 fb e5 93 15 0b 5e c0 c9 05 67 f0 f7 65 42 40 51 f8 6c 1d 1c 0b 5e c0 a2 b7 78 78 cf 65 42 40 3e 42 cd 90 2a 0b 5e c0 fc ab c7 7d ab 65 42 40 28 47 01 a2 60 0b 5e c0 13 60 58 fe 7c 65 42 40 a4 c5 19 c3 9c 0b 5e c0 12 bf 62 0d 17 65 42 40 ff 95 95 26 a5 0b 5e c0 3a 02 b8 59 bc 64 42 40 fb 3d b1 4e 95 0b 5e c0 f7 3c 7f da a8 64 42 40 8d b7 95 5e 9b 0b 5e c0 64 ca 87 a0 6a 64 42 40 6f 0c 01 c0 b1 0b 5e c0 8f 17 d2 e1 21 64 42 40 ca de 52 ce 17 0c 5e c0 a9 c0 c9 36 70 63 42 40 19 a9 f7 54 4e 0c 5e c0 d3 a4 14 74 7b 63 42 40 78 d3 2d 3b c4 0c 5e c0 df a8 15 a6 ef 63 42 40 c0 21 54 a9 d9 0c 5e c0 9e 95 b4 e2 1b 64 42 40 6b 99 0c c7 f3 0c 5e c0 16 be be d6 a5 64 42 40 e2 78 3e 03 ea 0c 5e c0 d9 5c 35 cf 11 65 42 40 25 3b 36 02 f1 0c 5e c0 2c a0 50 4f 1f 65 42 40 f9 65 30 46 24 0d 5e c0 c8 cd 70 03 3e 65 42 40 95 d6 df 12 80 0d 5e c0 ed f5 ee 8f f7 64 42 40 2d 98 f8 a3 a8 0d 5e c0 24 47 3a 03 23 65 42 40 d3 da 34 b6 d7 0d 5e c0 ce 6d c2 bd 32 65 42 40 cb 2c 42 b1 15 0e 5e c0 ea 3c 2a fe ef 64 42 40 76 50 89 eb 18 0e 5e c0 08 1d 74 09 87 64 42 40 ea 41 41 29 5a 0e 5e c0 aa 49 f0 86 34 64 42 40 ff 3e e3 c2 81 0e 5e c0 de e8 63 3e 20 64 42 40 52 61 6c 21 c8 0e 5e c0 df 6d de 38 29 64 42 40 43 37 fb 03 e5 0e 5e c0 88 67 09 32 02 64 42 40 cd 23 7f 30 f0 0e 5e c0 e7 fb a9 f1 d2 63 42 40 cd 57 c9 c7 ee 0e 5e c0 31 ed 9b fb ab 63 42 40 70 97 fd ba d3 0e 5e c0 de 8d 05 85 41 63 42 40 90 12 bb b6 b7 0e 5e c0 20 96 cd 1c 92 62 42 40 f4 a3 e1 94 b9 0e 5e c0 b4 71 c4 5a 7c 62 42 40 cf 9f 36 aa d3 0e 5e c0 71 ac 8b db 68 62 42 40 80 d7 67 ce fa 0e 5e c0 87 4f 3a 91 60 62 42 40 0f 0c 20 7c 28 0f 5e c0 02 66 be 83 9f 62 42 40 16 88 9e 94 49 0f 5e c0 bd 8f a3 39 b2 62 42 40 dc bc 71 52 98 0f 5e c0 01 32 74 ec a0 62 42 40 23 d7 4d 29 af 0f 5e c0 4a 97 fe 25 a9 62 42 40 84 bb b3 76 db 0f 5e c0 36 72 dd 94 f2 62 42 40 4c aa b6 9b e0 0f 5e c0 e9 65 14 cb 2d 63 42 40 1c 7d cc 07 04 10 5e c0 32 cb 9e 04 36 63 42 40 72 18 cc 5f 21 10 5e c0 4a d0 5f e8 11 63 42 40 e4 4a 3d 0b 42 10 5e c0 01 4f 5a b8 ac 62 42 40 39 7e a8 34 62 10 5e c0 e5 63 77 81 92 62 42 40 bd 01 66 be 83 10 5e c0 cf c0 c8 cb 9a 62 42 40 25 76 6d 6f b7 10 5e c0 93 70 21 8f e0 62 42 40 de 3a ff 76 d9 10 5e c0 c5 ad 82 18 e8 62 42 40 3d 65 35 5d 4f 11 5e c0 e9 7d e3 6b cf 62 42 40 24 25 3d 0c ad 11 5e c0 53 24 5f 09 a4 62 42 40 05 35 7c 0b eb 11 5e c0 d4 d6 88 60 1c 62 42 40 09 8d 60 e3 fa 11 5e c0 8b 71 fe 26 14 62 42 40 3a cd 02 ed 0e 12 5e c0 95 d4 09 68 22 62 42 40 75 02 9a 08 1b 12 5e c0 55 c1 a8 a4 4e 62 42 40 ad df 4c 4c 17 12 5e c0 bf d4 cf 9b 8a 62 42 40 26 8f a7 e5 07 12 5e c0 70 43 8c d7 bc 62 42 40 ef e5 3e 39 0a 12 5e c0 3d 29 93 1a da 62 42 40 d7 fa 22 a1 2d 12 5e c0 8c d6 51 d5 04 63 42 40 d1 91 5c fe 43 12 5e c0 34 f3 e4 9a 02 63 42 40 ba f7 70 c9 71 12 5e c0 ef 57 01 be db 62 42 40 15 e5 d2 f8 85 12 5e c0 cc 24 ea 05 9f 62 42 40 f2 42 3a 3c 84 12 5e c0 6a 13 27 f7 3b 62 42 40 6f f6 07 ca 6d 12 5e c0 43 1d 56 b8 e5 61 42 40 12 f5 82 4f 73 12 5e c0 4a eb 6f 09 c0 61 42 40 41 7f a1 47 8c 12 5e c0 c0 79 71 e2 ab 61 42 40 1e 4e 60 3a ad 12 5e c0 fb 94 63 b2 b8 61 42 40 9e 96 1f b8 ca 12 5e c0 75 ab e7 a4 f7 61 42 40 17 66 a1 9d d3 12 5e c0 57 06 d5 06 27 62 42 40 3f 8e e6 c8 ca 12 5e c0 f5 0f 22 19 72 62 42 40 93 e2 e3 13 b2 12 5e c0 e5 9d 43 19 aa 62 42 40 b1 e1 e9 95 b2 12 5e c0 70 43 8c d7 bc 62 42 40 ae 48 4c 50 c3 12 5e c0 2c f2 eb 87 d8 62 42 40 b2 4b 54 6f 0d 13 5e c0 c0 07 af 5d da 62 42 40 11 39 7d 3d 5f 13 5e c0 08 1d 74 09 87 62 42 40 b8 5b 92 03 76 13 5e c0 22 e1 7b 7f 83 62 42 40 77 2b 4b 74 96 13 5e c0 71 20 24 0b 98 62 42 40 92 74 cd e4 9b 13 5e c0 0c 03 96 5c c5 62 42 40 42 cd 90 2a 8a 13 5e c0 62 be bc 00 fb 62 42 40 ea 21 1a dd 41 13 5e c0 ef ff e3 84 09 63 42 40 59 8b 4f 01 30 13 5e c0 db 32 e0 2c 25 63 42 40 30 2c 7f be 2d 13 5e c0 e5 b8 53 3a 58 63 42 40 fb 07 91 0c 39 13 5e c0 2b a5 67 7a 89 63 42 40 47 92 20 5c 01 14 5e c0 f3 57 c8 5c 19 64 42 40 0d 50 1a 6a 14 14 5e c0 b3 d0 ce 69 16 64 42 40 5a d6 fd 63 21 14 5e c0 1d 03 b2 d7 bb 63 42 40 5b 44 14 93 37 14 5e c0 d2 fb c6 d7 9e 63 42 40 a5 4b ff 92 54 14 5e c0 a1 a1 7f 82 8b 63 42 40 d6 19 df 17 97 14 5e c0 42 99 46 93 8b 63 42 40 dd 61 13 99 b9 14 5e c0 37 fb 03 e5 b6 63 42 40 61 df 4e 22 c2 14 5e c0 8e b0 a8 88 d3 63 42 40 7d 76 c0 75 c5 14 5e c0 dc 46 03 78 0b 64 42 40 67 2b 2f f9 9f 14 5e c0 33 de 56 7a 6d 64 42 40 4a 97 fe 25 a9 14 5e c0 40 4b 57 b0 8d 64 42 40 c2 35 77 f4 bf 14 5e c0 64 5b 06 9c a5 64 42 40 20 09 fb 76 12 15 5e c0 d6 ab c8 e8 80 64 42 40 4f 1e 16 6a 4d 15 5e c0 8a c8 b0 8a 37 64 42 40 29 b2 d6 50 6a 15 5e c0 e5 47 fc 8a 35 64 42 40 d9 af 3b dd 79 15 5e c0 8e 40 bc ae 5f 64 42 40 35 7d 76 c0 75 15 5e c0 dc f3 fc 69 a3 64 42 40 bb 0e d5 94 64 15 5e c0 af 27 ba 2e fc 64 42 40 49 4d bb 98 66 15 5e c0 ef e1 92 e3 4e 65 42 40 25 7a 19 c5 72 15 5e c0 74 96 59 84 62 65 42 40 b6 2d ca 6c 90 15 5e c0 98 bd 6c 3b 6d 65 42 40 7b 2e 53 93 e0 15 5e c0 7a fc de a6 3f 65 42 40 21 20 5f 42 05 16 5e c0 c1 c5 8a 1a 4c 65 42 40 04 c6 fa 06 26 16 5e c0 77 82 fd d7 b9 65 42 40 38 6b f0 be 2a 16 5e c0 b2 4b 54 6f 0d 66 42 40 c7 7f 81 20 40 16 5e c0 79 74 23 2c 2a 66 42 40 38 4a 5e 9d 63 16 5e c0 f2 0c 1a fa 27 66 42 40 f6 29 c7 64 71 16 5e c0 9c dc ef 50 14 66 42 40 ff 21 fd f6 75 16 5e c0 d1 07 cb d8 d0 65 42 40 58 91 d1 01 49 16 5e c0 75 e8 f4 bc 1b 65 42 40 a6 d1 e4 62 0c 16 5e c0 ce 34 61 fb c9 64 42 40 90 f7 aa 95 09 16 5e c0 75 92 ad 2e a7 64 42 40 24 44 f9 82 16 16 5e c0 f6 60 52 7c 7c 64 42 40 f5 13 ce 6e 2d 16 5e c0 28 64 e7 6d 6c 64 42 40 30 a0 17 ee 5c 16 5e c0 f4 f8 bd 4d 7f 64 42 40 64 58 c5 1b 99 16 5e c0 19 c8 b3 cb b7 64 42 40 f4 8c 7d c9 c6 16 5e c0 17 49 bb d1 c7 64 42 40 7f a1 47 8c 9e 17 5e c0 5f 5e 80 7d 74 64 42 40 d6 ad 9e 93 de 17 5e c0 b9 6d df a3 fe 64 42 40 59 4f ad be ba 17 5e c0 f1 0e f0 a4 85 65 42 40 c3 7d e4 d6 a4 17 5e c0 59 18 22 a7 af 65 42 40 33 8c bb 41 b4 17 5e c0 95 2c 27 a1 f4 65 42 40 b6 2f a0 17 ee 17 5e c0 29 98 31 05 6b 66 42 40 b5 fb 55 80 ef 17 5e c0 66 bd 18 ca 89 66 42 40 6b 45 9b e3 dc 17 5e c0 3c f5 48 83 db 66 42 40 d4 2c d0 ee 90 17 5e c0 39 2a 37 51 4b 67 42 40 2b f9 d8 5d a0 17 5e c0 01 69 ff 03 ac 67 42 40 c2 86 a7 57 ca 17 5e c0 ac 8f 87 be bb 67 42 40 ad bf 25 00 ff 17 5e c0 78 25 c9 73 7d 67 42 40 f6 44 d7 85 1f 18 5e c0 70 98 68 90 82 67 42 40 75 3c 66 a0 32 18 5e c0 1f 82 aa d1 ab 67 42 40 9e cf 80 7a 33 18 5e c0 cd 76 85 3e 58 68 42 40 d1 77 b7 b2 44 18 5e c0 c9 03 91 45 9a 68 42 40 4c c4 5b e7 df 18 5e c0 1b d6 54 16 85 69 42 40 52 0c 90 68 02 19 5e c0 3d 10 59 a4 89 69 42 40 0a 68 22 6c 78 19 5e c0 01 30 9e 41 43 69 42 40 8b 4f 01 30 9e 19 5e c0 e2 ae 5e 45 46 69 42 40 41 bc ae 5f b0 19 5e c0 4e d3 67 07 5c 69 42 40 83 6d c4 93 dd 19 5e c0 8f aa 26 88 ba 69 42 40 4d 2c f0 15 dd 19 5e c0 b6 a0 f7 c6 10 6a 42 40 ee 26 f8 a6 e9 19 5e c0 a6 b9 15 c2 6a 6a 42 40 ff 76 d9 af 3b 1a 5e c0 08 03 cf bd 87 6b 42 40 24 26 a8 e1 5b 1a 5e c0 20 5f 42 05 87 6b 42 40 53 96 21 8e 75 1a 5e c0 de 02 09 8a 1f 6b 42 40 99 48 69 36 8f 1a 5e c0 e1 98 65 4f 02 6b 42 40 a9 85 92 c9 a9 1a 5e c0 50 53 cb d6 fa 6a 42 40 82 e5 08 19 c8 1a 5e c0 3d ba 11 16 15 6b 42 40 9c 17 27 be da 1a 5e c0 20 eb a9 d5 57 6b 42 40 68 5e 0e bb ef 1a 5e c0 0e d7 6a 0f 7b 6b 42 40 62 f5 47 18 06 1b 5e c0 3e 78 ed d2 86 6b 42 40 72 86 e2 8e 37 1b 5e c0 69 73 9c db 84 6b 42 40 28 f3 8f be 49 1b 5e c0 cb b9 14 57 95 6b 42 40 b2 9b 19 fd 68 1b 5e c0 a5 87 a1 d5 c9 6b 42 40 43 e1 b3 75 70 1b 5e c0 6e 52 d1 58 fb 6b 42 40 c9 74 e8 f4 bc 1b 5e c0 e6 22 be 13 b3 6c 42 40 19 1b ba d9 1f 1c 5e c0 c7 bd f9 0d 13 6d 42 40 18 5f b4 c7 0b 1c 5e c0 0d 35 0a 49 66 6d 42 40 db 6c ac c4 3c 1c 5e c0 76 c3 b6 45 99 6d 42 40 c6 4d 0d 34 9f 1c 5e c0 81 26 c2 86 a7 6d 42 40 ae 11 c1 38 b8 1c 5e c0 dc 9e 20 b1 dd 6d 42 40 ff e6 c5 89 af 1c 5e c0 00 00 00 00 00 6e 42 40 7a 52 26 35 b4 1c 5e c0 34 47 56 7e 19 6e 42 40 dd 7a 4d 0f 0a 1d 5e c0 de e8 63 3e 20 6e 42 40 0e db 16 65 36 1d 5e c0 7c 43 e1 b3 75 6e 42 40 50 e2 73 27 d8 1c 5e c0 08 ad 87 2f 13 6f 42 40 92 e8 65 14 cb 1c 5e c0 61 38 d7 30 43 6f 42 40 44 fd 2e 6c cd 1c 5e c0 a1 67 b3 ea 73 6f 42 40 32 03 95 f1 ef 1c 5e c0 50 51 f5 2b 9d 6f 42 40 f8 54 4e 7b 4a 1d 5e c0 87 a2 40 9f c8 6f 42 40 c2 33 a1 49 62 1d 5e c0 12 4e 0b 5e f4 6f 42 40 23 be 13 b3 5e 1d 5e c0 c8 96 e5 eb 32 70 42 40 b6 4b 1b 0e 4b 1d 5e c0 e8 dc ed 7a 69 70 42 40 4b 1e 4f cb 0f 1d 5e c0 5e 64 02 7e 8d 70 42 40 55 13 44 dd 07 1d 5e c0 ff cf 61 be bc 70 42 40 57 23 bb d2 32 1d 5e c0 7f 2e 1a 32 1e 71 42 40 6c 08 8e cb b8 1d 5e c0 c9 af 1f 62 83 71 42 40 90 4f c8 ce db 1d 5e c0 c9 cb 9a 58 e0 71 42 40 c5 90 9c 4c dc 1d 5e c0 0b 28 d4 d3 47 72 42 40 16 6d 8e 73 9b 1d 5e c0 ce 8a a8 89 3e 73 42 40 36 90 2e 36 ad 1d 5e c0 c2 da 18 3b e1 73 42 40 40 85 23 48 a5 1d 5e c0 cd 3d 24 7c ef 73 42 40 1b 82 e3 32 6e 1d 5e c0 d7 a0 2f bd fd 73 42 40 a6 0b b1 fa 23 1d 5e c0 75 5a b7 41 ed 73 42 40 69 72 31 06 d6 1c 5e c0 34 86 39 41 9b 74 42 40 8a 75 aa 7c cf 1c 5e c0 64 b2 b8 ff c8 74 42 40 80 d4 26 4e ee 1c 5e c0 af 44 a0 fa 07 75 42 40 42 76 de c6 66 1d 5e c0 3c f4 dd ad 2c 75 42 40 a8 e3 31 03 95 1d 5e c0 87 86 c5 a8 6b 75 42 40 a6 7b 9d d4 97 1d 5e c0 e6 b1 66 64 90 75 42 40 15 36 03 5c 90 1d 5e c0 9b 3b fa 5f ae 75 42 40 81 24 ec db 49 1d 5e c0 f8 89 03 e8 f7 75 42 40 10 ae 80 42 3d 1d 5e c0 a9 f8 bf 23 2a 76 42 40 06 85 41 99 46 1d 5e c0 f7 20 04 e4 4b 76 42 40 f7 e8 0d f7 91 1d 5e c0 bd e0 d3 9c bc 76 42 40 07 7d e9 ed cf 1d 5e c0 73 9c db 84 7b 77 42 40 a2 0b ea 5b e6 1d 5e c0 47 3e af 78 ea 77 42 40 1a fb 92 8d 07 1e 5e c0 cd c8 20 77 11 78 42 40 5f ed 28 ce 51 1e 5e c0 40 12 f6 ed 24 78 42 40 3d 0a d7 a3 70 1e 5e c0 02 84 0f 25 5a 78 42 40 70 23 65 8b a4 1e 5e c0 9f 91 08 8d 60 79 42 40 58 ad 4c f8 a5 1e 5e c0 cb a2 b0 8b a2 79 42 40 ba 82 6d c4 93 1e 5e c0 8d 41 27 84 0e 7a 42 40 1d ac ff 73 98 1e 5e c0 cc eb 88 43 36 7a 42 40 ca fc a3 6f d2 1e 5e c0 1a a4 e0 29 e4 7a 42 40 6e 6c 76 a4 fa 1e 5e c0 b9 fe 5d 9f 39 7b 42 40 29 3e 3e 21 3b 1f 5e c0 a6 5e b7 08 8c 7b 42 40 ce c1 33 a1 49 1f 5e c0 2d e9 28 07 b3 7b 42 40 05 6b 9c 4d 47 1f 5e c0 1b d5 e9 40 d6 7b 42 40 c3 10 39 7d 3d 1f 5e c0 77 f6 95 07 e9 7b 42 40 30 bb 27 0f 0b 1f 5e c0 e9 62 d3 4a 21 7c 42 40 aa 9e cc 3f fa 1e 5e c0 19 8f 52 09 4f 7c 42 40 5c e7 df 2e fb 1e 5e c0 39 b8 74 cc 79 7c 42 40 ce a6 23 80 9b 1f 5e c0 e3 c6 2d e6 e7 7c 42 40 73 2a 19 00 aa 1f 5e c0 36 8f c3 60 fe 7c 42 40 76 df 31 3c f6 1f 5e c0 dc 2a 88 81 ae 7d 42 40 b8 05 4b 75 01 20 5e c0 7d 96 e7 c1 dd 7d 42 40 59 31 5c 1d 00 20 5e c0 70 61 dd 78 77 7e 42 40 d4 b9 a2 94 10 20 5e c0 c9 e5 3f a4 df 7e 42 40 f3 57 c8 5c 19 20 5e c0 9c df 30 d1 20 7f 42 40 7d 3e ca 88 0b 20 5e c0 4c fd bc a9 48 7f 42 40 66 4a eb 6f 09 20 5e c0 dd ea 39 e9 7d 7f 42 40 b8 59 bc 58 18 20 5e c0 69 e1 b2 0a 9b 7f 42 40 92 23 9d 81 91 20 5e c0 0d e2 03 3b fe 7f 42 40 50 1d ab 94 9e 20 5e c0 06 48 34 81 22 80 42 40 67 2b 2f f9 9f 20 5e c0 9c 50 88 80 43 80 42 40 ad 35 94 da 8b 20 5e c0 20 45 9d b9 87 80 42 40 c4 77 62 d6 8b 20 5e c0 e1 b6 b6 f0 bc 80 42 40 57 cd 73 44 be 20 5e c0 17 d3 4c f7 3a 81 42 40 1e dd 08 8b 8a 21 5e c0 a0 e1 cd 1a bc 81 42 40 16 89 09 6a f8 21 5e c0 fc 8e e1 b1 9f 81 42 40 58 e3 6c 3a 02 22 5e c0 ea 7a a2 eb c2 81 42 40 a0 89 b0 e1 e9 21 5e c0 71 21 8f e0 46 82 42 40 c1 e0 9a 3b fa 21 5e c0 63 d0 09 a1 83 82 42 40 46 61 17 45 0f 22 5e c0 87 14 03 24 9a 82 42 40 35 0c 1f 11 53 22 5e c0 70 44 f7 ac 6b 82 42 40 06 10 3e 94 68 22 5e c0 da e3 85 74 78 82 42 40 ba c0 e5 b1 66 22 5e c0 e0 47 35 ec f7 82 42 40 f2 5e b5 32 e1 22 5e c0 1b 7f a2 b2 61 83 42 40 84 d8 99 42 e7 22 5e c0 14 ce 6e 2d 93 83 42 40 d6 73 d2 fb c6 22 5e c0 a2 25 8f a7 e5 83 42 40 a4 8e 8e ab 91 22 5e c0 2c ba f5 9a 1e 84 42 40 59 87 a3 ab 74 22 5e c0 17 10 5a 0f 5f 84 42 40 47 e5 26 6a 69 22 5e c0 10 23 84 47 1b 85 42 40 fd 4a e7 c3 b3 22 5e c0 70 23 65 8b a4 85 42 40 6f 7e c3 44 83 1f 5e c0 f8 aa 95 09 bf 8a 42 40 f3 90 29 1f 82 1e 5e c0 aa d4 ec 81 56 8c 42 40 1c 5c 3a e6 3c 1e 5e c0 0b 97 55 d8 0c 8c 42 40 3f 56 f0 db 10 1e 5e c0 f0 6a b9 33 13 8c 42 40 b0 01 11 e2 ca 1d 5e c0 87 a6 ec f4 83 8c 42 40 20 5f 42 05 87 1d 5e c0 a1 12 d7 31 ae 8c 42 40 a6 0e f2 7a 30 1d 5e c0 f7 b0 17 0a d8 8c 42 40 7f 32 c6 87 d9 1c 5e c0 d9 e8 9c 9f e2 8c 42 40 76 de c6 66 47 1c 5e c0 70 07 ea 94 47 8d 42 40 da a9 b9 dc 60 1b 5e c0 3a 91 60 aa 99 8d 42 40 1c 7a 8b 87 f7 1a 5e c0 37 c6 4e 78 09 8e 42 40 e2 76 68 58 8c 1a 5e c0 cf 87 67 09 32 8e 42 40 f3 e2 c4 57 3b 1a 5e c0 5d 31 23 bc 3d 8e 42 40 99 9e b0 c4 03 1a 5e c0 7c d5 ca 84 5f 8e 42 40 00 a9 4d 9c dc 19 5e c0 0d 54 c6 bf cf 8e 42 40 22 52 d3 2e a6 19 5e c0 73 63 7a c2 12 8f 42 40 f1 2a 6b 9b e2 18 5e c0 cc b3 92 56 7c 8f 42 40 b8 20 5b 96 af 18 5e c0 59 69 52 0a ba 8f 42 40 87 1a 85 24 b3 18 5e c0 eb 1c 03 b2 d7 8f 42 40 ad a3 aa 09 a2 18 5e c0 7e 38 48 88 f2 8f 42 40 b5 c6 a0 13 42 18 5e c0 9a 98 2e c4 ea 8f 42 40 fa d1 70 ca dc 17 5e c0 23 4c 51 2e 8d 8f 42 40 5a b8 ac c2 66 17 5e c0 13 d5 5b 03 5b 8f 42 40 6d 1d 1c ec 4d 17 5e c0 6b b8 c8 3d 5d 8f 42 40 79 01 f6 d1 a9 16 5e c0 57 ea 59 10 ca 8f 42 40 d4 0e 7f 4d d6 15 5e c0 01 17 64 cb f2 8f 42 40 67 ba d7 49 7d 15 5e c0 f3 cb 60 8c 48 90 42 40 f6 09 a0 18 59 15 5e c0 9f 94 49 0d 6d 90 42 40 cb 0f 5c e5 09 15 5e c0 5c 93 6e 4b e4 90 42 40 f6 0a 0b ee 07 15 5e c0 65 36 c8 24 23 91 42 40 00 e3 19 34 f4 14 5e c0 84 0e ba 84 43 91 42 40 0c 03 96 5c c5 14 5e c0 dc 65 bf ee 74 91 42 40 40 12 f6 ed 24 14 5e c0 a3 e5 40 0f b5 91 42 40 a2 25 8f a7 e5 13 5e c0 86 1c 5b cf 10 92 42 40 ba dd cb 7d 72 13 5e c0 22 8b 34 f1 0e 92 42 40 bf 0f 07 09 51 13 5e c0 d0 b4 c4 ca 68 92 42 40 9b 02 99 9d 45 13 5e c0 3d 0d 18 24 7d 92 42 40 06 2b 4e b5 16 13 5e c0 63 b8 3a 00 e2 92 42 40 af b0 e0 7e c0 12 5e c0 63 0e 82 8e 56 93 42 40 a6 b5 69 6c af 12 5e c0 6a 6d 1a db 6b 93 42 40 f6 9a 1e 14 94 12 5e c0 7b 12 d8 9c 83 93 42 40 8d 24 41 b8 02 12 5e c0 72 6e 13 ee 95 93 42 40 ea 5a 7b 9f aa 11 5e c0 27 33 de 56 7a 93 42 40 9d a1 b8 e3 4d 11 5e c0 88 68 74 07 b1 93 42 40 7b 68 1f 2b f8 10 5e c0 12 da 72 2e c5 93 42 40 18 0b 43 e4 f4 10 5e c0 a0 15 18 b2 ba 93 42 40 3a 05 f9 d9 c8 10 5e c0 76 a5 65 a4 de 93 42 40 4d 32 72 16 f6 0f 5e c0 ec a0 12 d7 31 94 42 40 ca 17 b4 90 80 0f 5e c0 90 85 e8 10 38 94 42 40 64 cb f2 75 19 0f 5e c0 78 63 41 61 50 94 42 40 3b a7 59 a0 dd 0e 5e c0 99 f4 f7 52 78 94 42 40 3b 53 e8 bc c6 0e 5e c0 12 4c 35 b3 96 94 42 40 0f ed 63 05 bf 0e 5e c0 6e 8a c7 45 b5 94 42 40 ad 35 94 da 8b 0e 5e c0 4d 15 8c 4a ea 94 42 40 c5 c9 fd 0e 45 0e 5e c0 67 47 aa ef fc 94 42 40 1d 1c ec 4d 0c 0e 5e c0 32 75 57 76 c1 94 42 40 ef e3 68 8e ac 0d 5e c0 73 12 4a 5f 08 95 42 40 90 dc 9a 74 5b 0d 5e c0 41 f2 ce a1 0c 95 42 40 20 98 a3 c7 ef 0c 5e c0 8e ca 4d d4 d2 94 42 40 42 0a 9e 42 ae 0c 5e c0 79 78 cf 81 e5 94 42 40 f8 16 d6 8d 77 0c 5e c0 a4 a7 c8 21 e2 94 42 40 41 bb 43 8a 01 0c 5e c0 2e 8f 35 23 83 94 42 40 52 b5 dd 04 df 0b 5e c0 f1 9c 2d 20 b4 94 42 40 03 97 c7 9a 91 0b 5e c0 ae f4 da 6c ac 94 42 40 a4 c6 84 98 4b 0b 5e c0 89 24 7a 19 c5 94 42 40 04 00 c7 9e 3d 0b 5e c0 5a f0 a2 af 20 95 42 40 40 12 f6 ed 24 0b 5e c0 9f 3a 56 29 3d 95 42 40 73 bb 97 fb e4 0a 5e c0 c2 dc ee e5 3e 95 42 40 f8 fb c5 6c c9 0a 5e c0 d4 b6 61 14 04 95 42 40 c2 15 50 a8 a7 0a 5e c0 46 07 24 61 df 94 42 40 37 6d c6 69 88 0a 5e c0 f3 8f be 49 d3 94 42 40 47 af 06 28 0d 0a 5e c0 d0 d5 56 ec 2f 95 42 40 b6 f2 92 ff c9 09 5e c0 80 b9 16 2d 40 95 42 40 bc 90 0e 0f 61 09 5e c0 2d b1 32 1a f9 94 42 40 a8 a7 8f c0 1f 09 5e c0 89 b5 f8 14 00 95 42 40 78 b4 71 c4 5a 08 5e c0 43 aa 28 5e 65 95 42 40 aa 0c e3 6e 10 08 5e c0 31 b4 3a 39 43 95 42 40 11 52 b7 b3 af 07 5e c0 82 38 0f 27 30 95 42 40 47 c8 40 9e 5d 07 5e c0 6e 31 3f 37 34 95 42 40 8d 09 31 97 54 07 5e c0 93 c6 68 1d 55 95 42 40 af 44 a0 fa 07 07 5e c0 2a 1a 6b 7f 67 95 42 40 c9 1f 0c 3c f7 06 5e c0 83 14 3c 85 5c 95 42 40 8b fd 65 f7 e4 06 5e c0 7e 74 ea ca 67 95 42 40 8f 18 3d b7 d0 06 5e c0 68 05 86 ac 6e 95 42 40 7e 74 ea ca 67 06 5e c0 9a 5e 62 2c d3 95 42 40 7a 8e c8 77 29 06 5e c0 ba d8 b4 52 08 96 42 40 e2 3c 9c c0 74 05 5e c0 c3 0d f8 fc 30 96 42 40 a8 aa d0 40 2c 05 5e c0 ed 2b 0f d2 53 96 42 40 89 61 87 31 e9 04 5e c0 00 e1 43 89 96 96 42 40 63 d3 4a 21 90 04 5e c0 95 81 03 5a ba 96 42 40 ac e2 8d cc 23 04 5e c0 ae 62 f1 9b c2 96 42 40 f8 a5 7e de 54 03 5e c0 67 d1 3b 15 70 97 42 40 68 ec 4b 36 1e 03 5e c0 b1 35 5b 79 c9 97 42 40 7b 4e 7a df f8 02 5e c0 4a 99 d4 d0 06 98 42 40 69 8f 17 d2 e1 00 5e c0 6a 6d 1a db 6b 9b 42 40 b4 92 56 7c 43 ff 5d c0 eb e1 cb 44 11 9e 42 40 a6 29 02 9c de fe 5d c0 21 21 ca 17 b4 9e 42 40 06 bb 61 db a2 fe 5d c0 67 81 76 87 14 9f 42 40 d7 88 60 1c 5c fe 5d c0 fd 88 5f b1 86 9f 42 40 ea 20 af 07 93 fc 5d c0 a2 98 bc 01 66 a2 42 40 d7 6b 7a 50 50 fc 5d c0 12 dd b3 ae d1 a2 42 40 28 0d 35 0a 49 fc 5d c0 71 02 d3 69 dd a2 42 40 b3 7d c8 5b ae fb 5d c0 45 68 04 1b d7 a3 42 40 24 44 f9 82 16 f8 5d c0 ca 6e 66 f4 a3 a9 42 40 4b 8f a6 7a 32 f7 5d c0 81 79 c8 94 0f ab 42 40 0a 81 5c e2 c8 f6 5d c0 0c cc 0a 45 ba ab 42 40 da aa 24 b2 0f f2 5d c0 72 c0 ae 26 4f b3 42 40 c1 01 2d 5d c1 f0 5d c0 e5 d5 39 06 64 b5 42 40 ad 31 e8 84 d0 ef 5d c0 16 fc 36 c4 78 b5 42 40 85 b2 f0 f5 b5 ee 5d c0 da c9 e0 28 79 b5 42 40 0d c7 f3 19 50 ed 5d c0 b0 02 7c b7 79 b5 42 40 d4 61 85 5b 3e ed 5d c0 80 7e df bf 79 b5 42 40 99 d7 11 87 6c ec 5d c0 74 d0 25 1c 7a b5 42 40 20 60 ad da 35 ec 5d c0 e5 43 50 35 7a b5 42 40 c5 1c 04 1d ad e9 5d c0 32 ad 4d 63 7b b5 42 40 13 ba 4b e2 ac e9 5d c0 2c d4 9a e6 1d b9 42 40 44 a6 7c 08 aa e9 5d c0 ac 74 77 9d 0d bb 42 40 f3 02 ec a3 53 e9 5d c0 22 a5 d9 3c 0e bb 42 40 be 2e c3 7f ba e8 5d c0 5e d7 2f d8 0d bb 42 40 0e 83 f9 2b 64 e7 5d c0 c7 bd f9 0d 13 bb 42 40 ca 8c b7 95 5e e7 5d c0 ee 79 fe b4 51 bf 42 40 9a 5c 8c 81 75 e5 5d c0 83 c3 0b 22 52 bf 42 40 d3 6a 48 dc 63 e5 5d c0 23 bb d2 32 52 bf 42 40 88 83 84 28 5f e5 5d c0 a9 4a 5b 5c e3 c3 42 40 e4 6a 64 57 5a e5 5d c0 e2 23 62 4a 24 c7 42 40 a3 95 7b 81 59 e5 5d c0 27 f8 a6 e9 b3 c7 42 40 82 ca f8 f7 19 d6 5d c0 66 2d 05 a4 fd df 42 40 29 e8 f6 92 c6 d3 5d c0 78 5f 95 0b 95 e3 42 40 6b ba 9e e8 ba d3 5d c0 b6 4b 1b 0e 4b e3 42 40 0b cf 4b c5 c6 d2 5d c0 cf f5 7d 38 48 e2 42 40 e8 15 4f 3d d2 d2 5d c0 51 69 c4 cc 3e e1 42 40 e1 b6 b6 f0 bc d2 5d c0 b5 35 22 18 07 e1 42 40 be 6c 3b 6d 8d d2 5d c0 19 02 80 63 cf e0 42 40 2f f7 c9 51 80 d2 5d c0 0b 44 4f ca a4 e0 42 40 a8 e0 f0 82 88 d2 5d c0 37 a9 68 ac fd df 42 40 4c e2 ac 88 9a d2 5d c0 e2 78 3e 03 ea df 42 40 6b f2 94 d5 74 d2 5d c0 fa 44 9e 24 5d df 42 40 a8 3a e4 66 b8 d1 5d c0 c2 dd 59 bb ed de 42 40 3e ed f0 d7 64 d1 5d c0 6d 1c b1 16 9f de 42 40 e4 13 b2 f3 36 d1 5d c0 6d 1c b1 16 9f de 42 40' +San Mateo X'00 00 00 00 03 05 00 00 00 56 0c 57 07 40 a1 5e c0 65 69 93 81 bd 8d 42 40 73 a0 87 da 36 85 5e c0 7d 3c f4 dd ad da 42 40 01 00 00 00 4f 01 00 00 00 00 00 00 a1 f6 5b 3b 51 9b 5e c0 49 2f 6a f7 ab da 42 40 16 88 9e 94 49 9b 5e c0 61 8b dd 3e ab da 42 40 20 7d 93 a6 41 9b 5e c0 a9 6b ed 7d aa da 42 40 80 b9 16 2d 40 9b 5e c0 67 7c 5f 5c aa da 42 40 a0 54 fb 74 3c 9b 5e c0 44 a6 7c 08 aa da 42 40 8b a8 89 3e 1f 9b 5e c0 b6 81 3b 50 a7 da 42 40 d9 e8 9c 9f e2 9a 5e c0 b6 81 3b 50 a7 da 42 40 13 9d 65 16 a1 9a 5e c0 1a df 17 97 aa da 42 40 d3 f8 85 57 92 9a 5e c0 b6 81 3b 50 a7 da 42 40 f7 b1 82 df 86 9a 5e c0 b6 81 3b 50 a7 da 42 40 e1 25 38 f5 81 9a 5e c0 b6 81 3b 50 a7 da 42 40 18 e9 45 ed 7e 9a 5e c0 ce a9 64 00 a8 da 42 40 a1 81 58 36 73 9a 5e c0 1a df 17 97 aa da 42 40 9c 50 88 80 43 9a 5e c0 09 a8 70 04 a9 da 42 40 29 76 34 0e f5 99 5e c0 b6 81 3b 50 a7 da 42 40 2a c4 23 f1 f2 99 5e c0 1b 47 ac c5 a7 da 42 40 4a 5f 08 39 ef 99 5e c0 45 da c6 9f a8 da 42 40 1c 96 06 7e 54 99 5e c0 1a df 17 97 aa da 42 40 f2 7d 71 a9 4a 99 5e c0 1a df 17 97 aa da 42 40 39 f0 6a b9 33 99 5e c0 b6 81 3b 50 a7 da 42 40 95 ee ae b3 21 99 5e c0 1a df 17 97 aa da 42 40 c7 40 b5 44 0c 99 5e c0 1a df 17 97 aa da 42 40 2f be 69 04 2a 99 5e c0 21 34 53 5c 90 da 42 40 c2 47 4d de ce 98 5e c0 e4 9d cd 6c eb d6 42 40 b5 82 3c f5 55 98 5e c0 51 63 9d cf b9 d5 42 40 49 3c 12 61 f4 97 5e c0 b0 63 92 2b c3 d4 42 40 af 47 6f d3 09 98 5e c0 2d 3f e8 77 81 d3 42 40 d3 ea 98 5b 2f 98 5e c0 34 e0 b5 20 41 d3 42 40 6d 89 dd d8 ca 98 5e c0 03 be 9e c9 00 d3 42 40 a4 df 72 66 b5 98 5e c0 03 6a 5b ea 9e d1 42 40 72 28 46 54 6e 98 5e c0 6d 75 95 f2 4f d1 42 40 c6 c1 09 9d 63 97 5e c0 39 6c ad 98 27 d0 42 40 78 9d 96 66 bd 96 5e c0 eb 9d 98 72 d0 ce 42 40 5a d8 8c 04 f3 96 5e c0 5c 11 f2 51 3a ce 42 40 b5 7b ed 09 b4 97 5e c0 ee b7 d7 8d a5 ce 42 40 8b a4 45 4b e4 97 5e c0 88 ac 7a 1b 90 ce 42 40 9a dd 55 29 2f 98 5e c0 18 b8 a2 25 a6 cd 42 40 ea 0e d5 14 3a 98 5e c0 83 24 b9 05 84 cd 42 40 41 8e f4 c7 0a 98 5e c0 1c 4e 56 73 40 cd 42 40 67 57 7a d3 0d 97 5e c0 3b 8a 22 16 d7 cb 42 40 99 da 8b f7 fa 96 5e c0 4b 0f c2 a2 d5 cb 42 40 e1 6b f5 90 5e 95 5e c0 09 aa b0 e9 b5 cb 42 40 b5 96 14 ce 54 94 5e c0 54 d8 45 78 a1 cb 42 40 c4 c2 f1 45 2f 94 5e c0 42 7b 38 83 2b cb 42 40 9d 05 89 a2 34 94 5e c0 2f 1e 2b 8e b5 ca 42 40 29 24 ad c8 93 93 5e c0 c5 f7 5b 78 a9 c9 42 40 98 d8 43 51 6f 93 5e c0 0a bb ca 0c a5 c9 42 40 c3 66 54 40 02 93 5e c0 9f dd 6d d4 97 c9 42 40 f3 e3 53 26 73 92 5e c0 9d 58 ef 7b 86 c9 42 40 6c cb 3e 64 d5 90 5e c0 af 46 de 54 54 c9 42 40 5b 4b 2d 0a d0 90 5e c0 66 8d cc ae 53 c9 42 40 1f d6 20 7c c6 90 5e c0 cb 86 e3 19 48 c9 42 40 99 d3 96 17 72 90 5e c0 bf 46 8b ce e1 c8 42 40 5a a1 8b 1a 1f 90 5e c0 de a1 f3 36 7d c8 42 40 4f 7f cf 3a 0f 90 5e c0 f8 16 c3 b4 5a c8 42 40 36 3f a5 bc f8 8f 5e c0 79 1d 09 cf 29 c8 42 40 d8 93 d2 c8 a3 8f 5e c0 3c b8 3f 21 71 c7 42 40 b0 64 cd e4 8a 8f 5e c0 4a d2 eb a9 50 c7 42 40 34 42 ec f0 5c 8f 5e c0 a1 78 bc b9 14 c7 42 40 47 98 c7 c3 01 8f 5e c0 8c 25 c7 cc 9d c6 42 40 6b 65 d0 6c 69 8e 5e c0 3b 99 94 18 d7 c5 42 40 be d3 77 1b 4a 8e 5e c0 7b 2e 2d 3f ae c5 42 40 fc 38 17 82 b6 8d 5e c0 35 6f eb b9 ed c4 42 40 f4 43 06 fa 94 8c 5e c0 a1 1b 16 d5 c2 c4 42 40 83 26 0b 2b 7a 8c 5e c0 dc e4 96 47 e0 c2 42 40 30 7a d6 de c7 8a 5e c0 3b 14 ef c4 87 c0 42 40 be 56 16 9a 70 8a 5e c0 5e 82 cf 7d 5f c0 42 40 17 3f a9 c3 f6 89 5e c0 5c f3 37 42 27 c0 42 40 c8 82 b7 92 93 89 5e c0 64 eb 96 83 57 c0 42 40 2d e4 72 15 f8 88 5e c0 21 c0 b5 16 03 c1 42 40 3a b9 be f4 61 88 5e c0 45 b9 3d af 77 c0 42 40 bc 1c d6 56 2c 88 5e c0 6a 1f 15 ba 01 c0 42 40 24 6b f1 d9 0b 88 5e c0 2d e3 a1 9d 1f c0 42 40 88 78 0c 73 6e 87 5e c0 79 50 f9 6c b0 c0 42 40 7b 9f aa 42 03 87 5e c0 c8 23 b8 91 b2 bf 42 40 73 a0 87 da 36 85 5e c0 9d 64 ab cb 29 bd 42 40 b7 5f 3e 59 31 86 5e c0 f0 6d fa b3 1f bb 42 40 36 e9 b6 44 2e 86 5e c0 20 b6 f4 68 aa bb 42 40 ca fb 38 9a 23 87 5e c0 77 31 cd 74 af bb 42 40 03 b2 d7 bb 3f 87 5e c0 0e f3 e5 05 d8 bb 42 40 c2 33 a1 49 62 87 5e c0 20 b6 f4 68 aa bb 42 40 9a 0b 5c 1e 6b 87 5e c0 df a9 80 7b 9e bb 42 40 39 0b 7b da e1 87 5e c0 07 0b 27 69 fe ba 42 40 82 e5 08 19 c8 87 5e c0 c1 71 19 37 35 ba 42 40 c8 ce db d8 ec 87 5e c0 cb a1 45 b6 f3 b9 42 40 9d 2d 20 b4 1e 88 5e c0 ea 79 37 16 14 ba 42 40 07 b3 09 30 2c 88 5e c0 17 ba 12 81 ea b9 42 40 5c 3c bc e7 c0 88 5e c0 a3 07 3e 06 2b ba 42 40 4b b0 38 9c f9 88 5e c0 c0 b1 67 cf 65 ba 42 40 82 90 2c 60 02 89 5e c0 19 c9 1e a1 66 ba 42 40 13 0a 11 70 08 89 5e c0 ad fb c7 42 74 ba 42 40 e5 b9 be 0f 07 89 5e c0 b7 2a 89 ec 83 ba 42 40 35 ef 38 45 47 89 5e c0 69 e4 f3 8a a7 ba 42 40 17 2a ff 5a 5e 89 5e c0 47 02 0d 36 75 ba 42 40 49 9d 80 26 c2 89 5e c0 77 11 a6 28 97 ba 42 40 52 98 f7 38 d3 89 5e c0 43 ad 69 de 71 ba 42 40 ce 88 d2 de e0 89 5e c0 c7 b9 4d b8 57 ba 42 40 ff ae cf 9c f5 89 5e c0 0c 79 04 37 52 ba 42 40 a2 0e 2b dc f2 89 5e c0 4c 8c 65 fa 25 ba 42 40 a8 53 1e dd 08 8a 5e c0 36 cc d0 78 22 ba 42 40 02 bc 05 12 14 8a 5e c0 23 bf 7e 88 0d ba 42 40 f3 8e 53 74 24 8a 5e c0 86 1c 5b cf 10 ba 42 40 3a 3e 5a 9c 31 8a 5e c0 76 70 b0 37 31 ba 42 40 50 01 30 9e 41 8a 5e c0 53 ce 17 7b 2f ba 42 40 bc 3c 9d 2b 4a 8a 5e c0 f2 d2 4d 62 10 ba 42 40 08 ac 1c 5a 64 8a 5e c0 b1 34 f0 a3 1a ba 42 40 fd 68 38 65 6e 8a 5e c0 d0 7b 63 08 00 ba 42 40 dd b5 84 7c d0 8a 5e c0 cf db d8 ec 48 b9 42 40 4c 8d d0 cf d4 8a 5e c0 4d 4e ed 0c 53 b9 42 40 57 3e cb f3 e0 8a 5e c0 c8 b6 0c 38 4b b9 42 40 5d fe 43 fa ed 8a 5e c0 6b 7e fc a5 45 b9 42 40 9c 33 a2 b4 37 8b 5e c0 17 2c d5 05 bc b8 42 40 d6 e6 ff 55 47 8b 5e c0 ef 58 6c 93 8a b8 42 40 9d a1 b8 e3 4d 8b 5e c0 00 c4 5d bd 8a b8 42 40 76 fb ac 32 53 8b 5e c0 b2 d5 e5 94 80 b8 42 40 3f c6 dc b5 84 8b 5e c0 18 0a d8 0e 46 b8 42 40 b8 af 03 e7 8c 8b 5e c0 26 37 8a ac 35 b8 42 40 0b 2a aa 7e a5 8b 5e c0 b4 ac fb c7 42 b8 42 40 9b 3b fa 5f ae 8b 5e c0 2e 90 a0 f8 31 b8 42 40 d9 43 fb 58 c1 8b 5e c0 35 42 3f 53 af b7 42 40 7c 45 b7 5e d3 8b 5e c0 5b 5d 4e 09 88 b7 42 40 b5 1b 7d cc 07 8c 5e c0 3f 72 6b d2 6d b7 42 40 c4 5b e7 df 2e 8c 5e c0 74 ee 76 bd 34 b7 42 40 4a f0 86 34 2a 8c 5e c0 4e 7d 20 79 e7 b6 42 40 b7 28 b3 41 26 8c 5e c0 e4 c0 ab e5 ce b6 42 40 8a 94 66 f3 38 8c 5e c0 a4 fa ce 2f 4a b6 42 40 4a f0 86 34 2a 8c 5e c0 88 0f ec f8 2f b6 42 40 ca a7 c7 b6 0c 8c 5e c0 17 f4 de 18 02 b6 42 40 26 a6 0b b1 fa 8b 5e c0 ed 28 ce 51 47 b5 42 40 ee 60 c4 3e 01 8c 5e c0 18 b0 e4 2a 16 b5 42 40 c9 38 46 b2 47 8c 5e c0 99 67 25 ad f8 b4 42 40 1f 69 70 5b 5b 8c 5e c0 8b a9 f4 13 ce b4 42 40 60 ac 6f 60 72 8c 5e c0 fe 0b 04 01 32 b4 42 40 28 44 c0 21 54 8c 5e c0 ff cc 20 3e b0 b3 42 40 78 60 00 e1 43 8c 5e c0 c3 7f ba 81 02 b3 42 40 c9 38 46 b2 47 8c 5e c0 39 f1 d5 8e e2 b2 42 40 de 39 94 a1 2a 8c 5e c0 f1 ba 7e c1 6e b2 42 40 75 39 25 20 26 8c 5e c0 ed 7d aa 0a 0d b2 42 40 11 ab 3f c2 30 8c 5e c0 a8 18 e7 6f 42 b1 42 40 1f 69 70 5b 5b 8c 5e c0 54 57 3e cb f3 b0 42 40 93 ff c9 df bd 8c 5e c0 9d b9 87 84 ef af 42 40 9e 5f 94 a0 bf 8c 5e c0 b0 c7 44 4a b3 af 42 40 97 74 94 83 d9 8c 5e c0 01 c1 1c 3d 7e af 42 40 65 8e e5 5d f5 8c 5e c0 74 07 b1 33 85 ae 42 40 d2 ac 6c 1f f2 8c 5e c0 6a a5 10 c8 25 ae 42 40 fe 26 14 22 e0 8c 5e c0 06 2c b9 8a c5 ad 42 40 97 e3 15 88 9e 8c 5e c0 da fe 95 95 26 ad 42 40 6b 0f 7b a1 80 8c 5e c0 f7 94 9c 13 7b ac 42 40 b0 c8 af 1f 62 8c 5e c0 bb 63 b1 4d 2a ac 42 40 a9 f7 54 4e 7b 8b 5e c0 2f e1 d0 5b 3c aa 42 40 39 d6 c5 6d 34 8b 5e c0 b1 fb 8e e1 b1 a9 42 40 ec 34 d2 52 79 8b 5e c0 32 8f fc c1 c0 a9 42 40 97 8c 63 24 7b 8b 5e c0 f6 5c a6 26 c1 a9 42 40 e8 33 a0 de 8c 8b 5e c0 8b a6 b3 93 c1 a9 42 40 72 a8 df 85 ad 8b 5e c0 27 66 bd 18 ca a9 42 40 8a 72 69 fc c2 8b 5e c0 14 59 6b 28 b5 a9 42 40 06 63 44 a2 d0 8b 5e c0 a4 df be 0e 9c a9 42 40 05 32 3b 8b de 8b 5e c0 a9 c0 c9 36 70 a9 42 40 7c d6 35 5a 0e 8c 5e c0 6b d6 19 df 17 a9 42 40 a6 7f 49 2a 53 8c 5e c0 7a 72 4d 81 cc a8 42 40 d4 10 55 f8 33 8c 5e c0 23 bd a8 dd af a8 42 40 df 33 12 a1 11 8c 5e c0 ec 6b 5d 6a 84 a8 42 40 33 6b 29 20 ed 8b 5e c0 2b a3 91 cf 2b a8 42 40 55 85 06 62 d9 8b 5e c0 0c 06 d7 dc d1 a7 42 40 08 5a 81 21 ab 8b 5e c0 20 99 0e 9d 9e a7 42 40 fa d5 1c 20 98 8b 5e c0 6f bb d0 5c a7 a7 42 40 d0 ee 90 62 80 8b 5e c0 ea e8 b8 1a d9 a7 42 40 05 6e dd cd 53 8b 5e c0 8d f0 f6 20 04 a8 42 40 c7 4b 37 89 41 8b 5e c0 26 8f a7 e5 07 a8 42 40 aa 60 54 52 27 8b 5e c0 f6 0a 0b ee 07 a8 42 40 79 58 a8 35 cd 8a 5e c0 85 5e 7f 12 9f a7 42 40 b9 c2 bb 5c c4 8a 5e c0 2b 4e b5 16 66 a7 42 40 15 3c 85 5c a9 8a 5e c0 91 99 0b 5c 1e a7 42 40 1d 38 67 44 69 8a 5e c0 b1 c3 98 f4 f7 a6 42 40 22 8d 0a 9c 6c 8a 5e c0 bc b3 76 db 85 a6 42 40 cf 66 d5 e7 6a 8a 5e c0 f4 c4 73 b6 80 a6 42 40 27 d8 7f 9d 9b 8a 5e c0 a6 29 02 9c de a5 42 40 2d af 5c 6f 9b 8a 5e c0 c8 40 9e 5d be a5 42 40 1c 78 b5 dc 99 8a 5e c0 45 62 82 1a be a5 42 40 73 68 91 ed 7c 8a 5e c0 a5 30 ef 71 a6 a5 42 40 1f d7 86 8a 71 8a 5e c0 eb 8b 84 b6 9c a5 42 40 a3 03 92 b0 6f 8a 5e c0 5a 2f 86 72 a2 a5 42 40 41 45 d5 af 74 8a 5e c0 d9 3d 79 58 a8 a5 42 40 15 74 7b 49 63 8a 5e c0 06 84 d6 c3 97 a5 42 40 a1 47 8c 9e 5b 8a 5e c0 6e 19 70 96 92 a5 42 40 62 a3 ac df 4c 8a 5e c0 db 6b 41 ef 8d a5 42 40 13 7e a9 9f 37 8a 5e c0 13 9a 24 96 94 a5 42 40 1f d8 f1 5f 20 8a 5e c0 78 25 c9 73 7d a5 42 40 d3 a2 3e c9 1d 8a 5e c0 e4 0f 06 9e 7b a5 42 40 09 8a 1f 63 ee 89 5e c0 fb 3e 1c 24 44 a5 42 40 f5 48 83 db da 89 5e c0 82 1b 29 5b 24 a5 42 40 2b 84 d5 58 c2 89 5e c0 b1 c4 03 ca a6 a4 42 40 ad fa 5c 6d c5 89 5e c0 aa 0a 0d c4 b2 9f 42 40 b3 d1 39 3f c5 89 5e c0 e1 46 ca 16 49 9f 42 40 98 6e 12 83 c0 89 5e c0 65 54 19 c6 dd 9e 42 40 4c 36 1e 6c b1 89 5e c0 27 16 f8 8a 6e 9d 42 40 62 f6 b2 ed b4 89 5e c0 9d 64 ab cb 29 9d 42 40 10 58 39 b4 c8 89 5e c0 2c d7 db 66 2a 9c 42 40 1c 06 f3 57 c8 89 5e c0 27 db c0 1d a8 9b 42 40 28 f1 b9 13 ec 89 5e c0 49 0f 43 ab 93 9b 42 40 09 8a 1f 63 ee 89 5e c0 49 0f 43 ab 93 9b 42 40 32 8e 91 ec 11 8a 5e c0 49 0f 43 ab 93 9b 42 40 de 02 09 8a 1f 8a 5e c0 49 0f 43 ab 93 9b 42 40 50 fd 83 48 86 8b 5e c0 b1 a4 dc 7d 8e 9b 42 40 7f 50 17 29 94 8b 5e c0 d6 e2 53 00 8c 9b 42 40 48 c1 53 c8 95 8b 5e c0 89 45 0c 3b 8c 9b 42 40 2a 8e 03 af 96 8b 5e c0 a6 2a 6d 71 8d 9b 42 40 b8 74 cc 79 c6 8d 5e c0 dc 9f 8b 86 8c 9b 42 40 60 e8 11 a3 e7 8d 5e c0 13 49 f4 32 8a 9b 42 40 04 92 b0 6f 27 8e 5e c0 26 1c 7a 8b 87 9b 42 40 07 b3 09 30 2c 8e 5e c0 0e c0 06 44 88 9b 42 40 27 88 ba 0f 40 8e 5e c0 ee 3e c7 47 8b 9b 42 40 da 55 48 f9 49 8e 5e c0 5f 7e a7 c9 8c 9b 42 40 c7 b9 4d b8 57 8e 5e c0 96 5b 5a 0d 89 9b 42 40 c1 02 98 32 70 8e 5e c0 e6 e5 b0 fb 8e 9b 42 40 77 a0 4e 79 74 8e 5e c0 f2 5f 20 08 90 9b 42 40 b8 74 cc 79 c6 8e 5e c0 5f 7e a7 c9 8c 9b 42 40 a9 2e e0 65 86 8f 5e c0 fb 20 cb 82 89 9b 42 40 1a a2 0a 7f 86 8f 5e c0 13 80 7f 4a 95 9a 42 40 20 45 9d b9 87 8f 5e c0 3b e2 90 0d a4 99 42 40 af d1 72 a0 87 8f 5e c0 67 0b 08 ad 87 99 42 40 1a a2 0a 7f 86 8f 5e c0 c6 8a 1a 4c c3 98 42 40 5a b7 41 ed b7 8f 5e c0 43 56 b7 7a 4e 98 42 40 ad 68 73 9c db 8f 5e c0 91 27 49 d7 4c 98 42 40 02 61 a7 58 35 91 5e c0 f1 2f 82 c6 4c 98 42 40 b6 d9 58 89 79 92 5e c0 54 54 fd 4a e7 97 42 40 c9 21 e2 e6 54 94 5e c0 4b 93 52 d0 ed 97 42 40 71 1e 4e 60 3a 94 5e c0 81 cc ce a2 77 96 42 40 d0 5e 7d 3c f4 93 5e c0 eb 53 8e c9 e2 92 42 40 d8 f5 0b 76 c3 92 5e c0 e8 6b 96 cb 46 8f 42 40 8b 1b b7 98 9f 92 5e c0 44 4e 5f cf d7 8e 42 40 45 83 14 3c 85 92 5e c0 27 6a 69 6e 85 8e 42 40 45 0c cf bb c0 92 5e c0 65 69 93 81 bd 8d 42 40 ee 41 08 c8 97 93 5e c0 ba 67 5d a3 e5 8e 42 40 12 bf 62 0d 17 94 5e c0 d2 54 4f e6 1f 8f 42 40 9c a3 8e 8e ab 94 5e c0 82 ff ad 64 c7 8e 42 40 6f 62 48 4e 26 95 5e c0 37 fe 44 65 c3 8e 42 40 85 5b 3e 92 92 95 5e c0 cd 91 95 5f 06 8f 42 40 6b 83 13 d1 af 95 5e c0 9f cb d4 24 78 8f 42 40 38 be f6 cc 92 95 5e c0 7b bd fb e3 bd 90 42 40 9d a0 4d 0e 9f 95 5e c0 ae 2c d1 59 66 91 42 40 0a f8 35 92 04 96 5e c0 72 31 06 d6 71 92 42 40 4a 99 d4 d0 06 97 5e c0 d0 0d 4d d9 e9 93 42 40 45 81 3e 91 27 97 5e c0 61 6d 8c 9d f0 94 42 40 8a 76 15 52 7e 97 5e c0 af 98 11 de 1e 96 42 40 a7 96 ad f5 45 98 5e c0 e8 31 ca 33 2f 97 42 40 6b b6 f2 92 ff 98 5e c0 bd c7 99 26 6c 97 42 40 b8 58 51 83 69 99 5e c0 b7 9a 75 c6 f7 97 42 40 87 df 4d b7 ec 99 5e c0 f2 ec f2 ad 0f 99 42 40 5d a3 e5 40 0f 9a 5e c0 ee ce da 6d 17 9c 42 40 c5 a9 d6 c2 2c 9a 5e c0 c8 94 0f 41 d5 9c 42 40 45 bd e0 d3 9c 9a 5e c0 de 1d 19 ab cd 9d 42 40 05 15 55 bf d2 9a 5e c0 89 7b 2c 7d e8 9e 42 40 88 bc e5 ea c7 9a 5e c0 42 af 3f 89 cf 9f 42 40 26 8e 3c 10 59 9a 5e c0 3e eb 1a 2d 07 a2 42 40 84 f3 54 f4 0c 9a 5e c0 6d 3d 9f 7d 1c a6 42 40 7d 08 aa 46 af 99 5e c0 95 ba 64 1c 23 ab 42 40 81 26 c2 86 a7 99 5e c0 eb 73 b5 15 fb ad 42 40 16 68 77 48 31 9a 5e c0 4b b0 38 9c f9 af 42 40 11 f2 77 31 33 9a 5e c0 32 4b cf 70 fe af 42 40 ff 36 12 f6 ac 9a 5e c0 cb b0 12 5e 32 b1 42 40 e7 08 1b 79 c2 9a 5e c0 62 5c 42 c4 68 b1 42 40 b3 b6 29 1e 17 9b 5e c0 39 27 f6 d0 3e b2 42 40 5f 3a 02 de 0f 9c 5e c0 f0 21 f0 24 61 b6 42 40 0c 9d c0 e0 0f 9c 5e c0 9b 7f 9c 30 61 b6 42 40 43 aa 28 5e 65 9c 5e c0 26 fe 28 ea cc b7 42 40 3c dc 0e 0d 8b 9c 5e c0 ae 61 86 c6 13 bb 42 40 ce 91 60 9e a0 9c 5e c0 9f 56 d1 1f 9a bb 42 40 98 fc 4f fe ee 9c 5e c0 bd 35 b0 55 82 bd 42 40 61 20 0a 0b 90 9d 5e c0 da 7a 37 1b e9 be 42 40 37 8c 82 e0 f1 9d 5e c0 2b 6a 30 0d c3 bf 42 40 76 c1 e0 9a 3b 9e 5e c0 80 2b d9 b1 11 c0 42 40 e9 ba 06 5f 62 9e 5e c0 2e b7 e0 d0 ef bf 42 40 c5 ee 46 68 62 9e 5e c0 f8 14 cb c8 ef bf 42 40 ba 7f 92 ea 6a 9e 5e c0 76 e9 1f 59 e8 bf 42 40 89 37 36 6b f1 9e 5e c0 fb 5b 8c cd 72 bf 42 40 35 61 fb c9 18 9f 5e c0 60 21 73 65 50 bf 42 40 f9 87 2d 3d 9a 9f 5e c0 6f bd a6 07 05 bf 42 40 2a 90 d9 59 f4 9f 5e c0 19 af 79 55 67 bf 42 40 a9 d8 98 d7 11 a0 5e c0 30 08 12 77 bf bf 42 40 66 0c 47 ca 26 a0 5e c0 9c ff da 10 fe bf 42 40 02 13 c6 b9 7d a0 5e c0 b3 c3 f9 dc 01 c1 42 40 09 ec 21 5d ae a0 5e c0 24 71 80 36 93 c1 42 40 fd 4c bd 6e 11 a1 5e c0 5f 0c e5 44 bb c2 42 40 d7 a0 f6 c0 1b a1 5e c0 a9 88 f3 1f 2f c3 42 40 56 0c 57 07 40 a1 5e c0 a6 09 db 4f c6 c4 42 40 9a 3b 61 67 3d a1 5e c0 49 11 1b ee d3 c4 42 40 4b 1e 4f cb 0f a1 5e c0 50 a8 a7 8f c0 c5 42 40 cb d5 8f 4d f2 a0 5e c0 fa 08 fc e1 e7 c5 42 40 40 16 a2 43 e0 a0 5e c0 b2 49 7e c4 af c6 42 40 ce 1c 92 5a 28 a1 5e c0 ce fd d5 e3 be c9 42 40 12 fc 73 7a 1d a1 5e c0 69 07 8c e8 1c cb 42 40 a6 ba 80 97 19 a1 5e c0 5e 67 43 fe 99 cb 42 40 48 77 93 a7 c3 a0 5e c0 27 87 e2 e3 fb cb 42 40 db 06 df b7 7c a0 5e c0 22 cf d8 b2 4c cc 42 40 06 2b 4e b5 16 a0 5e c0 5c 3c bc e7 c0 cc 42 40 07 05 c6 6a d3 9f 5e c0 0d 7b b3 96 2e ce 42 40 b3 bf 20 8e d1 9f 5e c0 2a 82 f5 b4 38 ce 42 40 15 c7 81 57 cb 9f 5e c0 25 5a f2 78 5a ce 42 40 fd e7 99 e4 b2 9f 5e c0 b4 56 2e f5 9b d0 42 40 6d 1c b1 16 9f 9f 5e c0 5c 3d 27 bd 6f d2 42 40 41 29 4d 44 b4 9f 5e c0 24 8f e8 1a 09 d5 42 40 7e 85 7a 1f b5 9f 5e c0 5b de eb 00 24 d5 42 40 09 c7 e0 26 b8 9f 5e c0 fb d0 14 2a 83 d5 42 40 45 4b 1e 4f cb 9f 5e c0 95 f2 5a 09 dd d7 42 40 08 95 f6 a7 08 a0 5e c0 7e fe eb dc b4 d9 42 40 26 6f 24 e9 1e a0 5e c0 b5 2c 82 06 60 da 42 40 98 85 61 c2 27 a0 5e c0 0e b1 b0 14 a4 da 42 40 0f 09 df fb 1b 9f 5e c0 87 fd 9e 58 a7 da 42 40 33 c2 db 83 10 9f 5e c0 87 fd 9e 58 a7 da 42 40 57 ea 59 10 ca 9e 5e c0 87 fd 9e 58 a7 da 42 40 ae 67 08 c7 2c 9e 5e c0 ea 5a 7b 9f aa da 42 40 83 4f 73 f2 22 9e 5e c0 87 fd 9e 58 a7 da 42 40 a0 a9 d7 2d 02 9e 5e c0 87 fd 9e 58 a7 da 42 40 ae d6 89 cb f1 9d 5e c0 87 fd 9e 58 a7 da 42 40 d2 8f 86 53 e6 9d 5e c0 87 fd 9e 58 a7 da 42 40 92 eb a6 94 d7 9d 5e c0 23 a0 c2 11 a4 da 42 40 8b fb 8f 4c 87 9d 5e c0 87 fd 9e 58 a7 da 42 40 04 54 38 82 54 9d 5e c0 87 fd 9e 58 a7 da 42 40 28 7c b6 0e 0e 9d 5e c0 87 fd 9e 58 a7 da 42 40 a9 33 f7 90 f0 9c 5e c0 23 a0 c2 11 a4 da 42 40 85 e9 7b 0d c1 9c 5e c0 87 fd 9e 58 a7 da 42 40 80 bb ec d7 9d 9c 5e c0 87 fd 9e 58 a7 da 42 40 8d b7 95 5e 9b 9c 5e c0 87 fd 9e 58 a7 da 42 40 9e 09 4d 12 4b 9c 5e c0 87 fd 9e 58 a7 da 42 40 10 94 db f6 3d 9c 5e c0 ea 5a 7b 9f aa da 42 40 97 aa b4 c5 35 9c 5e c0 ea 5a 7b 9f aa da 42 40 bb 63 b1 4d 2a 9c 5e c0 ea 5a 7b 9f aa da 42 40 04 8f 6f ef 1a 9c 5e c0 f7 3c 7f da a8 da 42 40 18 d1 76 4c dd 9b 5e c0 23 a0 c2 11 a4 da 42 40 34 2b db 87 bc 9b 5e c0 87 fd 9e 58 a7 da 42 40 d8 9b 18 92 93 9b 5e c0 b6 81 3b 50 a7 da 42 40 68 59 f7 8f 85 9b 5e c0 68 b0 a9 f3 a8 da 42 40 67 80 0b b2 65 9b 5e c0 7d 3c f4 dd ad da 42 40 a1 f6 5b 3b 51 9b 5e c0 49 2f 6a f7 ab da 42 40' diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java index fdb07b5d4007..afa681cde33b 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveStorageFormats.java @@ -303,6 +303,8 @@ public void verifyDataProviderCompleteness() .filter(format -> !"CSV".equals(format)) // REGEX is read-only .filter(format -> !"REGEX".equals(format)) + // ESRI is read-only + .filter(format -> !"ESRI".equals(format)) // TODO when using JSON serde Hive fails with ClassNotFoundException: org.apache.hive.hcatalog.data.JsonSerDe .filter(format -> !"JSON".equals(format)) // OPENX is not supported in Hive by default